From da9d7df04330bd0586682e819c5c21615baf42b8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Apr 2024 13:48:25 -0700 Subject: [PATCH 01/30] feat: Enable columnar shuffle by default --- .../scala/org/apache/comet/CometConf.scala | 16 ++++++------- core/src/execution/datafusion/planner.rs | 23 +++++++++++++++++++ .../spark/sql/CometTPCDSQuerySuite.scala | 3 ++- 3 files changed, 33 insertions(+), 9 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index ca4bf47098..fda906d034 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -142,14 +142,14 @@ object CometConf { .booleanConf .createWithDefault(false) - val COMET_COLUMNAR_SHUFFLE_ENABLED: ConfigEntry[Boolean] = conf( - "spark.comet.columnar.shuffle.enabled") - .doc( - "Force Comet to only use columnar shuffle for CometScan and Spark regular operators. " + - "If this is enabled, Comet native shuffle will not be enabled but only Arrow shuffle. " + - "By default, this config is false.") - .booleanConf - .createWithDefault(false) + val COMET_COLUMNAR_SHUFFLE_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.columnar.shuffle.enabled") + .doc( + "Whether to enable Arrow-based columnar shuffle for Comet and Spark regular operators. " + + "If this is enabled, Comet prefers columnar shuffle than native shuffle. " + + "By default, this config is true.") + .booleanConf + .createWithDefault(true) val COMET_EXEC_BROADCAST_FORCE_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_EXEC_CONFIG_PREFIX.broadcast.enabled") diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index 72174790b2..958505a921 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -1039,6 +1039,21 @@ impl PhysicalPlanner { .collect(); let full_schema = Arc::new(Schema::new(all_fields)); + // Because we cast dictionary array to array in scan operator, + // we need to change dictionary type to data type for join filter expression. + let fields: Vec<_> = full_schema + .fields() + .iter() + .map(|f| match f.data_type() { + DataType::Dictionary(_, val_type) => { + Arc::new(Field::new(f.name(), val_type.as_ref().clone(), true)) + } + _ => f.clone(), + }) + .collect(); + + let full_schema = Arc::new(Schema::new(fields)); + let physical_expr = self.create_expr(expr, full_schema)?; let (left_field_indices, right_field_indices) = expr_to_columns(&physical_expr, left_fields.len(), right_fields.len())?; @@ -1057,6 +1072,14 @@ impl PhysicalPlanner { .into_iter() .map(|i| right.schema().field(i).clone()), ) + // Because we cast dictionary array to array in scan operator, + // we need to change dictionary type to data type for join filter expression. + .map(|f| match f.data_type() { + DataType::Dictionary(_, val_type) => { + Field::new(f.name(), val_type.as_ref().clone(), true) + } + _ => f.clone(), + }) .collect_vec(); let filter_schema = Schema::new_with_metadata(filter_fields, HashMap::new()); diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala index cdbd7194dc..7a07f86296 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala @@ -152,8 +152,9 @@ class CometTPCDSQuerySuite conf.set(CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ALL_EXPR_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key, "true") + conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "20g") conf.set(MEMORY_OFFHEAP_ENABLED.key, "true") - conf.set(MEMORY_OFFHEAP_SIZE.key, "2g") + conf.set(MEMORY_OFFHEAP_SIZE.key, "20g") conf } From 2ae45fb6eea22a4dfbb185188cb80078e83f5dd4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 9 Apr 2024 15:29:22 -0700 Subject: [PATCH 02/30] Update plan stability --- .../approved-plans-v1_4/q1/explain.txt | 298 +++-- .../approved-plans-v1_4/q1/simplified.txt | 72 +- .../approved-plans-v1_4/q10/explain.txt | 140 +- .../approved-plans-v1_4/q10/simplified.txt | 134 +- .../approved-plans-v1_4/q11/explain.txt | 582 ++++---- .../approved-plans-v1_4/q11/simplified.txt | 180 +-- .../approved-plans-v1_4/q12/explain.txt | 114 +- .../approved-plans-v1_4/q12/simplified.txt | 16 +- .../approved-plans-v1_4/q13/explain.txt | 112 +- .../approved-plans-v1_4/q13/simplified.txt | 100 +- .../approved-plans-v1_4/q14a/explain.txt | 781 +++++------ .../approved-plans-v1_4/q14a/simplified.txt | 250 ++-- .../approved-plans-v1_4/q14b/explain.txt | 678 +++++----- .../approved-plans-v1_4/q14b/simplified.txt | 370 +++--- .../approved-plans-v1_4/q15/explain.txt | 94 +- .../approved-plans-v1_4/q15/simplified.txt | 76 +- .../approved-plans-v1_4/q16/explain.txt | 118 +- .../approved-plans-v1_4/q16/simplified.txt | 122 +- .../approved-plans-v1_4/q17/explain.txt | 138 +- .../approved-plans-v1_4/q17/simplified.txt | 126 +- .../approved-plans-v1_4/q18/explain.txt | 136 +- .../approved-plans-v1_4/q18/simplified.txt | 122 +- .../approved-plans-v1_4/q19/explain.txt | 100 +- .../approved-plans-v1_4/q19/simplified.txt | 94 +- .../approved-plans-v1_4/q2/explain.txt | 202 +-- .../approved-plans-v1_4/q2/simplified.txt | 105 +- .../approved-plans-v1_4/q20/explain.txt | 114 +- .../approved-plans-v1_4/q20/simplified.txt | 16 +- .../approved-plans-v1_4/q21/explain.txt | 92 +- .../approved-plans-v1_4/q21/simplified.txt | 78 +- .../approved-plans-v1_4/q22/explain.txt | 96 +- .../approved-plans-v1_4/q22/simplified.txt | 78 +- .../approved-plans-v1_4/q23a/explain.txt | 706 +++++----- .../approved-plans-v1_4/q23a/simplified.txt | 295 ++--- .../approved-plans-v1_4/q23b/explain.txt | 881 ++++++------ .../approved-plans-v1_4/q23b/simplified.txt | 349 +++-- .../approved-plans-v1_4/q24a/explain.txt | 294 ++-- .../approved-plans-v1_4/q24a/simplified.txt | 100 +- .../approved-plans-v1_4/q24b/explain.txt | 294 ++-- .../approved-plans-v1_4/q24b/simplified.txt | 100 +- .../approved-plans-v1_4/q25/explain.txt | 160 +-- .../approved-plans-v1_4/q25/simplified.txt | 132 +- .../approved-plans-v1_4/q26/explain.txt | 110 +- .../approved-plans-v1_4/q26/simplified.txt | 96 +- .../approved-plans-v1_4/q27/explain.txt | 110 +- .../approved-plans-v1_4/q27/simplified.txt | 96 +- .../approved-plans-v1_4/q28/explain.txt | 340 ++--- .../approved-plans-v1_4/q28/simplified.txt | 140 +- .../approved-plans-v1_4/q29/explain.txt | 186 +-- .../approved-plans-v1_4/q29/simplified.txt | 140 +- .../approved-plans-v1_4/q3/explain.txt | 64 +- .../approved-plans-v1_4/q3/simplified.txt | 58 +- .../approved-plans-v1_4/q30/explain.txt | 322 ++--- .../approved-plans-v1_4/q30/simplified.txt | 88 +- .../approved-plans-v1_4/q31/explain.txt | 808 +++++------ .../approved-plans-v1_4/q31/simplified.txt | 300 +++-- .../approved-plans-v1_4/q32/explain.txt | 164 +-- .../approved-plans-v1_4/q32/simplified.txt | 96 +- .../approved-plans-v1_4/q33/explain.txt | 395 +++--- .../approved-plans-v1_4/q33/simplified.txt | 190 ++- .../approved-plans-v1_4/q34/explain.txt | 160 ++- .../approved-plans-v1_4/q34/simplified.txt | 108 +- .../approved-plans-v1_4/q35/explain.txt | 138 +- .../approved-plans-v1_4/q35/simplified.txt | 132 +- .../approved-plans-v1_4/q36/explain.txt | 130 +- .../approved-plans-v1_4/q36/simplified.txt | 16 +- .../approved-plans-v1_4/q37/explain.txt | 90 +- .../approved-plans-v1_4/q37/simplified.txt | 88 +- .../approved-plans-v1_4/q38/explain.txt | 238 ++-- .../approved-plans-v1_4/q38/simplified.txt | 160 +-- .../approved-plans-v1_4/q39a/explain.txt | 208 +-- .../approved-plans-v1_4/q39a/simplified.txt | 152 ++- .../approved-plans-v1_4/q39b/explain.txt | 208 +-- .../approved-plans-v1_4/q39b/simplified.txt | 152 ++- .../approved-plans-v1_4/q4/explain.txt | 898 +++++++------ .../approved-plans-v1_4/q4/simplified.txt | 272 ++-- .../approved-plans-v1_4/q40/explain.txt | 110 +- .../approved-plans-v1_4/q40/simplified.txt | 94 +- .../approved-plans-v1_4/q41/explain.txt | 64 +- .../approved-plans-v1_4/q41/simplified.txt | 56 +- .../approved-plans-v1_4/q42/explain.txt | 64 +- .../approved-plans-v1_4/q42/simplified.txt | 58 +- .../approved-plans-v1_4/q43/explain.txt | 64 +- .../approved-plans-v1_4/q43/simplified.txt | 58 +- .../approved-plans-v1_4/q44/explain.txt | 18 +- .../approved-plans-v1_4/q44/simplified.txt | 6 +- .../approved-plans-v1_4/q45/explain.txt | 122 +- .../approved-plans-v1_4/q45/simplified.txt | 108 +- .../approved-plans-v1_4/q46/explain.txt | 172 +-- .../approved-plans-v1_4/q46/simplified.txt | 86 +- .../approved-plans-v1_4/q47/explain.txt | 283 ++-- .../approved-plans-v1_4/q47/simplified.txt | 42 +- .../approved-plans-v1_4/q48/explain.txt | 100 +- .../approved-plans-v1_4/q48/simplified.txt | 84 +- .../approved-plans-v1_4/q49/explain.txt | 596 +++++---- .../approved-plans-v1_4/q49/simplified.txt | 244 ++-- .../approved-plans-v1_4/q5/explain.txt | 312 +++-- .../approved-plans-v1_4/q5/simplified.txt | 224 ++-- .../approved-plans-v1_4/q50/explain.txt | 106 +- .../approved-plans-v1_4/q50/simplified.txt | 84 +- .../approved-plans-v1_4/q51/explain.txt | 335 ++--- .../approved-plans-v1_4/q51/simplified.txt | 136 +- .../approved-plans-v1_4/q52/explain.txt | 64 +- .../approved-plans-v1_4/q52/simplified.txt | 58 +- .../approved-plans-v1_4/q53/explain.txt | 136 +- .../approved-plans-v1_4/q53/simplified.txt | 16 +- .../approved-plans-v1_4/q54/explain.txt | 368 ++--- .../approved-plans-v1_4/q54/simplified.txt | 112 +- .../approved-plans-v1_4/q55/explain.txt | 64 +- .../approved-plans-v1_4/q55/simplified.txt | 58 +- .../approved-plans-v1_4/q56/explain.txt | 395 +++--- .../approved-plans-v1_4/q56/simplified.txt | 190 ++- .../approved-plans-v1_4/q57/explain.txt | 283 ++-- .../approved-plans-v1_4/q57/simplified.txt | 42 +- .../approved-plans-v1_4/q58/explain.txt | 458 ++++--- .../approved-plans-v1_4/q58/simplified.txt | 166 +-- .../approved-plans-v1_4/q59/explain.txt | 256 ++-- .../approved-plans-v1_4/q59/simplified.txt | 41 +- .../approved-plans-v1_4/q6/explain.txt | 190 +-- .../approved-plans-v1_4/q6/simplified.txt | 138 +- .../approved-plans-v1_4/q60/explain.txt | 395 +++--- .../approved-plans-v1_4/q60/simplified.txt | 190 ++- .../approved-plans-v1_4/q61/explain.txt | 318 ++--- .../approved-plans-v1_4/q61/simplified.txt | 170 +-- .../approved-plans-v1_4/q62/explain.txt | 86 +- .../approved-plans-v1_4/q62/simplified.txt | 80 +- .../approved-plans-v1_4/q63/explain.txt | 136 +- .../approved-plans-v1_4/q63/simplified.txt | 16 +- .../approved-plans-v1_4/q64/explain.txt | 586 ++++---- .../approved-plans-v1_4/q64/simplified.txt | 430 +++--- .../approved-plans-v1_4/q65/explain.txt | 300 +++-- .../approved-plans-v1_4/q65/simplified.txt | 68 +- .../approved-plans-v1_4/q66/explain.txt | 300 +++-- .../approved-plans-v1_4/q66/simplified.txt | 153 ++- .../approved-plans-v1_4/q67/explain.txt | 128 +- .../approved-plans-v1_4/q67/simplified.txt | 16 +- .../approved-plans-v1_4/q68/explain.txt | 172 +-- .../approved-plans-v1_4/q68/simplified.txt | 86 +- .../approved-plans-v1_4/q69/explain.txt | 138 +- .../approved-plans-v1_4/q69/simplified.txt | 130 +- .../approved-plans-v1_4/q7/explain.txt | 110 +- .../approved-plans-v1_4/q7/simplified.txt | 96 +- .../approved-plans-v1_4/q70/explain.txt | 228 ++-- .../approved-plans-v1_4/q70/simplified.txt | 64 +- .../approved-plans-v1_4/q71/explain.txt | 136 +- .../approved-plans-v1_4/q71/simplified.txt | 14 +- .../approved-plans-v1_4/q72/explain.txt | 231 ++-- .../approved-plans-v1_4/q72/simplified.txt | 182 ++- .../approved-plans-v1_4/q73/explain.txt | 160 ++- .../approved-plans-v1_4/q73/simplified.txt | 108 +- .../approved-plans-v1_4/q74/explain.txt | 574 ++++---- .../approved-plans-v1_4/q74/simplified.txt | 180 +-- .../approved-plans-v1_4/q75/explain.txt | 850 ++++++------ .../approved-plans-v1_4/q75/simplified.txt | 372 +++--- .../approved-plans-v1_4/q76/explain.txt | 98 +- .../approved-plans-v1_4/q76/simplified.txt | 106 +- .../approved-plans-v1_4/q77/explain.txt | 714 +++++----- .../approved-plans-v1_4/q77/simplified.txt | 290 ++-- .../approved-plans-v1_4/q78/explain.txt | 508 +++---- .../approved-plans-v1_4/q78/simplified.txt | 144 +- .../approved-plans-v1_4/q79/explain.txt | 136 +- .../approved-plans-v1_4/q79/simplified.txt | 74 +- .../approved-plans-v1_4/q8/explain.txt | 206 +-- .../approved-plans-v1_4/q8/simplified.txt | 142 +- .../approved-plans-v1_4/q80/explain.txt | 426 +++--- .../approved-plans-v1_4/q80/simplified.txt | 294 ++-- .../approved-plans-v1_4/q81/explain.txt | 314 ++--- .../approved-plans-v1_4/q81/simplified.txt | 88 +- .../approved-plans-v1_4/q82/explain.txt | 90 +- .../approved-plans-v1_4/q82/simplified.txt | 88 +- .../approved-plans-v1_4/q83/explain.txt | 430 +++--- .../approved-plans-v1_4/q83/simplified.txt | 158 +-- .../approved-plans-v1_4/q85/explain.txt | 144 +- .../approved-plans-v1_4/q85/simplified.txt | 126 +- .../approved-plans-v1_4/q86/explain.txt | 116 +- .../approved-plans-v1_4/q86/simplified.txt | 16 +- .../approved-plans-v1_4/q87/explain.txt | 238 ++-- .../approved-plans-v1_4/q87/simplified.txt | 160 +-- .../approved-plans-v1_4/q88/explain.txt | 1178 +++++++++-------- .../approved-plans-v1_4/q88/simplified.txt | 288 ++-- .../approved-plans-v1_4/q89/explain.txt | 134 +- .../approved-plans-v1_4/q89/simplified.txt | 16 +- .../approved-plans-v1_4/q9/explain.txt | 30 +- .../approved-plans-v1_4/q9/simplified.txt | 10 +- .../approved-plans-v1_4/q90/explain.txt | 238 ++-- .../approved-plans-v1_4/q90/simplified.txt | 120 +- .../approved-plans-v1_4/q91/explain.txt | 142 +- .../approved-plans-v1_4/q91/simplified.txt | 14 +- .../approved-plans-v1_4/q92/explain.txt | 164 +-- .../approved-plans-v1_4/q92/simplified.txt | 96 +- .../approved-plans-v1_4/q93/explain.txt | 76 +- .../approved-plans-v1_4/q93/simplified.txt | 62 +- .../approved-plans-v1_4/q94/explain.txt | 118 +- .../approved-plans-v1_4/q94/simplified.txt | 122 +- .../approved-plans-v1_4/q95/explain.txt | 150 ++- .../approved-plans-v1_4/q95/simplified.txt | 178 +-- .../approved-plans-v1_4/q96/explain.txt | 72 +- .../approved-plans-v1_4/q96/simplified.txt | 68 +- .../approved-plans-v1_4/q97/explain.txt | 151 +-- .../approved-plans-v1_4/q97/simplified.txt | 88 +- .../approved-plans-v1_4/q98/explain.txt | 138 +- .../approved-plans-v1_4/q98/simplified.txt | 88 +- .../approved-plans-v1_4/q99/explain.txt | 86 +- .../approved-plans-v1_4/q99/simplified.txt | 80 +- .../approved-plans-v2_7/q10a/explain.txt | 136 +- .../approved-plans-v2_7/q10a/simplified.txt | 136 +- .../approved-plans-v2_7/q11/explain.txt | 574 ++++---- .../approved-plans-v2_7/q11/simplified.txt | 180 +-- .../approved-plans-v2_7/q12/explain.txt | 114 +- .../approved-plans-v2_7/q12/simplified.txt | 16 +- .../approved-plans-v2_7/q14/explain.txt | 678 +++++----- .../approved-plans-v2_7/q14/simplified.txt | 370 +++--- .../approved-plans-v2_7/q14a/explain.txt | 1048 ++++++++------- .../approved-plans-v2_7/q14a/simplified.txt | 476 ++++--- .../approved-plans-v2_7/q18a/explain.txt | 647 ++++----- .../approved-plans-v2_7/q18a/simplified.txt | 402 +++--- .../approved-plans-v2_7/q20/explain.txt | 114 +- .../approved-plans-v2_7/q20/simplified.txt | 16 +- .../approved-plans-v2_7/q22/explain.txt | 94 +- .../approved-plans-v2_7/q22/simplified.txt | 76 +- .../approved-plans-v2_7/q22a/explain.txt | 275 ++-- .../approved-plans-v2_7/q22a/simplified.txt | 138 +- .../approved-plans-v2_7/q24/explain.txt | 310 ++--- .../approved-plans-v2_7/q24/simplified.txt | 128 +- .../approved-plans-v2_7/q27a/explain.txt | 319 ++--- .../approved-plans-v2_7/q27a/simplified.txt | 212 +-- .../approved-plans-v2_7/q34/explain.txt | 160 ++- .../approved-plans-v2_7/q34/simplified.txt | 108 +- .../approved-plans-v2_7/q35/explain.txt | 138 +- .../approved-plans-v2_7/q35/simplified.txt | 132 +- .../approved-plans-v2_7/q35a/explain.txt | 134 +- .../approved-plans-v2_7/q35a/simplified.txt | 134 +- .../approved-plans-v2_7/q36a/explain.txt | 247 ++-- .../approved-plans-v2_7/q36a/simplified.txt | 126 +- .../approved-plans-v2_7/q47/explain.txt | 283 ++-- .../approved-plans-v2_7/q47/simplified.txt | 42 +- .../approved-plans-v2_7/q49/explain.txt | 596 +++++---- .../approved-plans-v2_7/q49/simplified.txt | 244 ++-- .../approved-plans-v2_7/q51a/explain.txt | 653 ++++----- .../approved-plans-v2_7/q51a/simplified.txt | 201 ++- .../approved-plans-v2_7/q57/explain.txt | 283 ++-- .../approved-plans-v2_7/q57/simplified.txt | 42 +- .../approved-plans-v2_7/q5a/explain.txt | 719 +++++----- .../approved-plans-v2_7/q5a/simplified.txt | 256 ++-- .../approved-plans-v2_7/q6/explain.txt | 190 +-- .../approved-plans-v2_7/q6/simplified.txt | 138 +- .../approved-plans-v2_7/q64/explain.txt | 586 ++++---- .../approved-plans-v2_7/q64/simplified.txt | 430 +++--- .../approved-plans-v2_7/q67a/explain.txt | 487 ++++--- .../approved-plans-v2_7/q67a/simplified.txt | 130 +- .../approved-plans-v2_7/q70a/explain.txt | 335 +++-- .../approved-plans-v2_7/q70a/simplified.txt | 170 ++- .../approved-plans-v2_7/q72/explain.txt | 231 ++-- .../approved-plans-v2_7/q72/simplified.txt | 182 ++- .../approved-plans-v2_7/q74/explain.txt | 574 ++++---- .../approved-plans-v2_7/q74/simplified.txt | 180 +-- .../approved-plans-v2_7/q75/explain.txt | 850 ++++++------ .../approved-plans-v2_7/q75/simplified.txt | 372 +++--- .../approved-plans-v2_7/q77a/explain.txt | 861 ++++++------ .../approved-plans-v2_7/q77a/simplified.txt | 144 +- .../approved-plans-v2_7/q78/explain.txt | 508 +++---- .../approved-plans-v2_7/q78/simplified.txt | 144 +- .../approved-plans-v2_7/q80a/explain.txt | 533 ++++---- .../approved-plans-v2_7/q80a/simplified.txt | 102 +- .../approved-plans-v2_7/q86a/explain.txt | 233 ++-- .../approved-plans-v2_7/q86a/simplified.txt | 108 +- .../approved-plans-v2_7/q98/explain.txt | 130 +- .../approved-plans-v2_7/q98/simplified.txt | 84 +- 268 files changed, 29471 insertions(+), 27963 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt index b0ea6bed8b..acab82fdec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt @@ -1,44 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Filter (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_returns (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (24) - : : +- * Filter (23) - : : +- * HashAggregate (22) - : : +- Exchange (21) - : : +- * HashAggregate (20) - : : +- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * ColumnarToRow (13) - : : : +- CometFilter (12) - : : : +- CometScan parquet spark_catalog.default.store_returns (11) - : : +- ReusedExchange (14) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.store (27) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer (34) +TakeOrderedAndProject (44) ++- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * ColumnarToRow (12) + : : : +- CometFilter (11) + : : : +- CometHashAggregate (10) + : : : +- CometColumnarExchange (9) + : : : +- RowToColumnar (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_returns (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (28) + : : +- * ColumnarToRow (27) + : : +- CometFilter (26) + : : +- CometHashAggregate (25) + : : +- CometColumnarExchange (24) + : : +- CometHashAggregate (23) + : : +- CometHashAggregate (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * ColumnarToRow (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store_returns (13) + : : +- ReusedExchange (16) + : +- BroadcastExchange (35) + : +- * ColumnarToRow (34) + : +- CometProject (33) + : +- CometFilter (32) + : +- CometScan parquet spark_catalog.default.store (31) + +- BroadcastExchange (41) + +- * ColumnarToRow (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.customer (38) (unknown) Scan parquet spark_catalog.default.store_returns @@ -56,7 +60,7 @@ Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) (3) ColumnarToRow [codegen id : 2] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -(4) ReusedExchange [Reuses operator id: 45] +(4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 2] @@ -76,199 +80,203 @@ Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum#7] Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -(8) Exchange +(8) RowToColumnar Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) HashAggregate [codegen id : 9] +(9) CometColumnarExchange +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(10) CometHashAggregate Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] -Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(10) Filter [codegen id : 9] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] -Condition : isnotnull(ctr_total_return#12) +(11) CometFilter +Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] +Condition : isnotnull(ctr_total_return#11) + +(12) ColumnarToRow [codegen id : 8] +Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] (unknown) Scan parquet spark_catalog.default.store_returns Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#12)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(12) CometFilter +(14) CometFilter Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Condition : isnotnull(sr_store_sk#2) -(13) ColumnarToRow [codegen id : 4] +(15) ColumnarToRow [codegen id : 4] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -(14) ReusedExchange [Reuses operator id: 45] +(16) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#6] -(15) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [sr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(16) Project [codegen id : 4] +(18) Project [codegen id : 4] Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -(17) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 4] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum#14] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] +Aggregate Attributes [1]: [sum#13] +Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] -(18) Exchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(20) RowToColumnar +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] -(19) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] +(21) CometColumnarExchange +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(22) CometHashAggregate +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] -Results [2]: [sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] - -(20) HashAggregate [codegen id : 5] -Input [2]: [ctr_store_sk#11, ctr_total_return#12] -Keys [1]: [ctr_store_sk#11] -Functions [1]: [partial_avg(ctr_total_return#12)] -Aggregate Attributes [2]: [sum#16, count#17] -Results [3]: [ctr_store_sk#11, sum#18, count#19] - -(21) Exchange -Input [3]: [ctr_store_sk#11, sum#18, count#19] -Arguments: hashpartitioning(ctr_store_sk#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(22) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#11, sum#18, count#19] -Keys [1]: [ctr_store_sk#11] -Functions [1]: [avg(ctr_total_return#12)] -Aggregate Attributes [1]: [avg(ctr_total_return#12)#20] -Results [2]: [(avg(ctr_total_return#12)#20 * 1.2) AS (avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11 AS ctr_store_sk#11#22] - -(23) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#21) - -(24) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] + +(23) CometHashAggregate +Input [2]: [ctr_store_sk#10, ctr_total_return#11] +Keys [1]: [ctr_store_sk#10] +Functions [1]: [partial_avg(ctr_total_return#11)] + +(24) CometColumnarExchange +Input [3]: [ctr_store_sk#10, sum#15, count#16] +Arguments: hashpartitioning(ctr_store_sk#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(25) CometHashAggregate +Input [3]: [ctr_store_sk#10, sum#15, count#16] +Keys [1]: [ctr_store_sk#10] +Functions [1]: [avg(ctr_total_return#11)] + +(26) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#17, ctr_store_sk#10#18] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#17) + +(27) ColumnarToRow [codegen id : 5] +Input [2]: [(avg(ctr_total_return) * 1.2)#17, ctr_store_sk#10#18] + +(28) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#17, ctr_store_sk#10#18] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [ctr_store_sk#11#22] +(29) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ctr_store_sk#10] +Right keys [1]: [ctr_store_sk#10#18] Join type: Inner -Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#21) +Join condition: (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#17) -(26) Project [codegen id : 9] -Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] -Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] +(30) Project [codegen id : 8] +Output [2]: [ctr_customer_sk#9, ctr_store_sk#10] +Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#17, ctr_store_sk#10#18] (unknown) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_state#24] +Output [2]: [s_store_sk#19, s_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [s_store_sk#23, s_state#24] -Condition : ((isnotnull(s_state#24) AND (s_state#24 = TN)) AND isnotnull(s_store_sk#23)) +(32) CometFilter +Input [2]: [s_store_sk#19, s_state#20] +Condition : ((isnotnull(s_state#20) AND (s_state#20 = TN)) AND isnotnull(s_store_sk#19)) -(29) CometProject -Input [2]: [s_store_sk#23, s_state#24] -Arguments: [s_store_sk#23], [s_store_sk#23] +(33) CometProject +Input [2]: [s_store_sk#19, s_state#20] +Arguments: [s_store_sk#19], [s_store_sk#19] -(30) ColumnarToRow [codegen id : 7] -Input [1]: [s_store_sk#23] +(34) ColumnarToRow [codegen id : 6] +Input [1]: [s_store_sk#19] -(31) BroadcastExchange -Input [1]: [s_store_sk#23] +(35) BroadcastExchange +Input [1]: [s_store_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [s_store_sk#23] +(36) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ctr_store_sk#10] +Right keys [1]: [s_store_sk#19] Join type: Inner Join condition: None -(33) Project [codegen id : 9] -Output [1]: [ctr_customer_sk#10] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#23] +(37) Project [codegen id : 8] +Output [1]: [ctr_customer_sk#9] +Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#19] (unknown) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#25, c_customer_id#26] +Output [2]: [c_customer_sk#21, c_customer_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [c_customer_sk#25, c_customer_id#26] -Condition : isnotnull(c_customer_sk#25) +(39) CometFilter +Input [2]: [c_customer_sk#21, c_customer_id#22] +Condition : isnotnull(c_customer_sk#21) -(36) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#25, c_customer_id#26] +(40) ColumnarToRow [codegen id : 7] +Input [2]: [c_customer_sk#21, c_customer_id#22] -(37) BroadcastExchange -Input [2]: [c_customer_sk#25, c_customer_id#26] +(41) BroadcastExchange +Input [2]: [c_customer_sk#21, c_customer_id#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_customer_sk#10] -Right keys [1]: [c_customer_sk#25] +(42) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ctr_customer_sk#9] +Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(39) Project [codegen id : 9] -Output [1]: [c_customer_id#26] -Input [3]: [ctr_customer_sk#10, c_customer_sk#25, c_customer_id#26] +(43) Project [codegen id : 8] +Output [1]: [c_customer_id#22] +Input [3]: [ctr_customer_sk#9, c_customer_sk#21, c_customer_id#22] -(40) TakeOrderedAndProject -Input [1]: [c_customer_id#26] -Arguments: 100, [c_customer_id#26 ASC NULLS FIRST], [c_customer_id#26] +(44) TakeOrderedAndProject +Input [1]: [c_customer_id#22] +Arguments: 100, [c_customer_id#22 ASC NULLS FIRST], [c_customer_id#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#27] +Output [2]: [d_date_sk#6, d_year#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [2]: [d_date_sk#6, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2000)) AND isnotnull(d_date_sk#6)) +(46) CometFilter +Input [2]: [d_date_sk#6, d_year#23] +Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2000)) AND isnotnull(d_date_sk#6)) -(43) CometProject -Input [2]: [d_date_sk#6, d_year#27] +(47) CometProject +Input [2]: [d_date_sk#6, d_year#23] Arguments: [d_date_sk#6], [d_date_sk#6] -(44) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(45) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 13 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index 6d4c0fca7a..c8f2ec9eb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -1,45 +1,47 @@ TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (9) + WholeStageCodegen (8) Project [c_customer_id] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk] BroadcastHashJoin [ctr_store_sk,s_store_sk] Project [ctr_customer_sk,ctr_store_sk] BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - InputAdapter - Exchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [sr_store_sk,sr_customer_sk] - CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 + ColumnarToRow + InputAdapter + CometFilter [ctr_total_return] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum] + CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_store_sk,sr_customer_sk] + CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (6) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_store_sk,sum,count] - InputAdapter - Exchange [ctr_store_sk] #4 - WholeStageCodegen (5) - HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] - InputAdapter - Exchange [sr_customer_sk,sr_store_sk] #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [(avg(ctr_total_return) * 1.2)] + CometHashAggregate [ctr_store_sk,sum,count] + CometColumnarExchange [ctr_store_sk] #4 + CometHashAggregate [ctr_store_sk,ctr_total_return] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum] + CometColumnarExchange [sr_customer_sk,sr_store_sk] #5 + RowToColumnar WholeStageCodegen (4) HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] Project [sr_customer_sk,sr_store_sk,sr_return_amt] @@ -53,7 +55,7 @@ TakeOrderedAndProject [c_customer_id] ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [s_store_sk] @@ -61,7 +63,7 @@ TakeOrderedAndProject [c_customer_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) + WholeStageCodegen (7) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 1ea234408b..a99a75e9a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -1,47 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer_demographics (34) +* ColumnarToRow (45) ++- CometTakeOrderedAndProject (44) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) + +- RowToColumnar (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * Filter (25) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer_demographics (34) (unknown) Scan parquet spark_catalog.default.customer @@ -68,7 +70,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) ReusedExchange [Reuses operator id: 48] +(6) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] (7) BroadcastHashJoin [codegen id : 2] @@ -101,7 +103,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(13) ReusedExchange [Reuses operator id: 48] +(13) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#13] (14) BroadcastHashJoin [codegen id : 4] @@ -134,7 +136,7 @@ ReadSchema: struct (19) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -(20) ReusedExchange [Reuses operator id: 48] +(20) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 6] @@ -232,50 +234,54 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -(41) Exchange +(41) RowToColumnar Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(42) HashAggregate [codegen id : 10] +(42) CometColumnarExchange +Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] +Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(43) CometHashAggregate Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] -(43) TakeOrderedAndProject -Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] -Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] +(44) CometTakeOrderedAndProject +Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#21 ASC NULLS FIRST,cd_marital_status#22 ASC NULLS FIRST,cd_education_status#23 ASC NULLS FIRST,cd_purchase_estimate#24 ASC NULLS FIRST,cd_credit_rating#25 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[cd_gender#21,cd_marital_status#22,cd_education_status#23,cnt1#31,cd_purchase_estimate#24,cnt2#32,cd_credit_rating#25,cnt3#33,cd_dep_count#26,cnt4#34,cd_dep_employed_count#27,cnt5#35,cd_dep_college_count#28,cnt6#36]), 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] + +(45) ColumnarToRow [codegen id : 10] +Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#38, d_moy#39] +Output [3]: [d_date_sk#9, d_year#37, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [3]: [d_date_sk#9, d_year#38, d_moy#39] -Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 1)) AND (d_moy#39 <= 4)) AND isnotnull(d_date_sk#9)) +(47) CometFilter +Input [3]: [d_date_sk#9, d_year#37, d_moy#38] +Condition : (((((isnotnull(d_year#37) AND isnotnull(d_moy#38)) AND (d_year#37 = 2002)) AND (d_moy#38 >= 1)) AND (d_moy#38 <= 4)) AND isnotnull(d_date_sk#9)) -(46) CometProject -Input [3]: [d_date_sk#9, d_year#38, d_moy#39] +(48) CometProject +Input [3]: [d_date_sk#9, d_year#37, d_moy#38] Arguments: [d_date_sk#9], [d_date_sk#9] -(47) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(48) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 89893c831e..8923131021 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -1,75 +1,77 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - InputAdapter - Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow +WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] + CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_county,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index 64f486f71e..a24e545fbc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt @@ -1,76 +1,84 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) - : :- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (51) - : +- * Filter (50) - : +- * HashAggregate (49) - : +- Exchange (48) - : +- * HashAggregate (47) - : +- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (43) - : : +- * BroadcastHashJoin Inner BuildRight (42) - : : :- * ColumnarToRow (37) - : : : +- CometFilter (36) - : : : +- CometScan parquet spark_catalog.default.customer (35) - : : +- BroadcastExchange (41) - : : +- * ColumnarToRow (40) - : : +- CometFilter (39) - : : +- CometScan parquet spark_catalog.default.web_sales (38) - : +- ReusedExchange (44) - +- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * ColumnarToRow (56) - : : +- CometFilter (55) - : : +- CometScan parquet spark_catalog.default.customer (54) - : +- BroadcastExchange (60) - : +- * ColumnarToRow (59) - : +- CometFilter (58) - : +- CometScan parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (63) +TakeOrderedAndProject (80) ++- * Project (79) + +- * BroadcastHashJoin Inner BuildRight (78) + :- * Project (59) + : +- * BroadcastHashJoin Inner BuildRight (58) + : :- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometColumnarExchange (15) + : : : +- RowToColumnar (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (36) + : : +- * ColumnarToRow (35) + : : +- CometHashAggregate (34) + : : +- CometColumnarExchange (33) + : : +- RowToColumnar (32) + : : +- * HashAggregate (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * Project (27) + : : : +- * BroadcastHashJoin Inner BuildRight (26) + : : : :- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.store_sales (22) + : : +- ReusedExchange (28) + : +- BroadcastExchange (57) + : +- * ColumnarToRow (56) + : +- CometFilter (55) + : +- CometHashAggregate (54) + : +- CometColumnarExchange (53) + : +- RowToColumnar (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * ColumnarToRow (41) + : : : +- CometFilter (40) + : : : +- CometScan parquet spark_catalog.default.customer (39) + : : +- BroadcastExchange (45) + : : +- * ColumnarToRow (44) + : : +- CometFilter (43) + : : +- CometScan parquet spark_catalog.default.web_sales (42) + : +- ReusedExchange (48) + +- BroadcastExchange (77) + +- * ColumnarToRow (76) + +- CometHashAggregate (75) + +- CometColumnarExchange (74) + +- RowToColumnar (73) + +- * HashAggregate (72) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * ColumnarToRow (62) + : : +- CometFilter (61) + : : +- CometScan parquet spark_catalog.default.customer (60) + : +- BroadcastExchange (66) + : +- * ColumnarToRow (65) + : +- CometFilter (64) + : +- CometScan parquet spark_catalog.default.web_sales (63) + +- ReusedExchange (69) (unknown) Scan parquet spark_catalog.default.customer @@ -116,7 +124,7 @@ Join condition: None Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(10) ReusedExchange [Reuses operator id: 76] +(10) ReusedExchange [Reuses operator id: 84] Output [2]: [d_date_sk#14, d_year#15] (11) BroadcastHashJoin [codegen id : 3] @@ -136,300 +144,316 @@ Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discoun Aggregate Attributes [1]: [sum#16] Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -(14) Exchange +(14) RowToColumnar Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 16] +(15) CometColumnarExchange +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] -Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] -(16) Filter [codegen id : 16] -Input [2]: [customer_id#19, year_total#20] -Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) +(17) CometFilter +Input [2]: [customer_id#18, year_total#19] +Condition : (isnotnull(year_total#19) AND (year_total#19 > 0.00)) + +(18) ColumnarToRow [codegen id : 16] +Input [2]: [customer_id#18, year_total#19] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Output [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter -Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) +(20) CometFilter +Input [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_customer_id#21)) -(19) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +(21) ColumnarToRow [codegen id : 6] +Input [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#33)] +PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -Condition : isnotnull(ss_customer_sk#29) +(23) CometFilter +Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Condition : isnotnull(ss_customer_sk#28) -(22) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +(24) ColumnarToRow [codegen id : 4] +Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -(23) BroadcastExchange -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +(25) BroadcastExchange +Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#21] -Right keys [1]: [ss_customer_sk#29] +(26) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#20] +Right keys [1]: [ss_customer_sk#28] Join type: Inner Join condition: None -(25) Project [codegen id : 6] -Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +(27) Project [codegen id : 6] +Output [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Input [12]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -(26) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#34, d_year#35] +(28) ReusedExchange [Reuses operator id: 88] +Output [2]: [d_date_sk#33, d_year#34] -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#34] +(29) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#31] +Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] -Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] - -(29) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] -Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] -Aggregate Attributes [1]: [sum#36] -Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] - -(30) Exchange -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] -Results [3]: [c_customer_id#22 AS customer_id#38, c_preferred_cust_flag#25 AS customer_preferred_cust_flag#39, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#40] - -(32) BroadcastExchange -Input [3]: [customer_id#38, customer_preferred_cust_flag#39, year_total#40] +(30) Project [codegen id : 6] +Output [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] +Input [12]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31, d_date_sk#33, d_year#34] + +(31) HashAggregate [codegen id : 6] +Input [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] +Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] +Aggregate Attributes [1]: [sum#35] +Results [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] + +(32) RowToColumnar +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] + +(33) CometColumnarExchange +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] +Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometHashAggregate +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] +Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] + +(35) ColumnarToRow [codegen id : 7] +Input [3]: [customer_id#37, customer_preferred_cust_flag#38, year_total#39] + +(36) BroadcastExchange +Input [3]: [customer_id#37, customer_preferred_cust_flag#38, year_total#39] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#38] +(37) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#37] Join type: Inner Join condition: None -(34) Project [codegen id : 16] -Output [4]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40] -Input [5]: [customer_id#19, year_total#20, customer_id#38, customer_preferred_cust_flag#39, year_total#40] +(38) Project [codegen id : 16] +Output [4]: [customer_id#18, year_total#19, customer_preferred_cust_flag#38, year_total#39] +Input [5]: [customer_id#18, year_total#19, customer_id#37, customer_preferred_cust_flag#38, year_total#39] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Output [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(36) CometFilter -Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] -Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_customer_id#42)) +(40) CometFilter +Input [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] +Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) -(37) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +(41) ColumnarToRow [codegen id : 10] +Input [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Output [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#53)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#52)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(39) CometFilter -Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -Condition : isnotnull(ws_bill_customer_sk#49) +(43) CometFilter +Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_customer_sk#48) -(40) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +(44) ColumnarToRow [codegen id : 8] +Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] -(41) BroadcastExchange -Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +(45) BroadcastExchange +Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(42) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#41] -Right keys [1]: [ws_bill_customer_sk#49] +(46) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#40] +Right keys [1]: [ws_bill_customer_sk#48] Join type: Inner Join condition: None -(43) Project [codegen id : 10] -Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -Input [12]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +(47) Project [codegen id : 10] +Output [10]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Input [12]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] -(44) ReusedExchange [Reuses operator id: 76] -Output [2]: [d_date_sk#54, d_year#55] +(48) ReusedExchange [Reuses operator id: 84] +Output [2]: [d_date_sk#53, d_year#54] -(45) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#54] +(49) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#53] Join type: Inner Join condition: None -(46) Project [codegen id : 10] -Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] -Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52, d_date_sk#54, d_year#55] - -(47) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] -Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] -Aggregate Attributes [1]: [sum#56] -Results [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] - -(48) Exchange -Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] -Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(49) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] -Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58] -Results [2]: [c_customer_id#42 AS customer_id#59, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58,18,2) AS year_total#60] - -(50) Filter [codegen id : 11] -Input [2]: [customer_id#59, year_total#60] -Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.00)) - -(51) BroadcastExchange -Input [2]: [customer_id#59, year_total#60] +(50) Project [codegen id : 10] +Output [10]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#54] +Input [12]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51, d_date_sk#53, d_year#54] + +(51) HashAggregate [codegen id : 10] +Input [10]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#54] +Keys [8]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#50 - ws_ext_discount_amt#49)))] +Aggregate Attributes [1]: [sum#55] +Results [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#56] + +(52) RowToColumnar +Input [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#56] + +(53) CometColumnarExchange +Input [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#56] +Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(54) CometHashAggregate +Input [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#56] +Keys [8]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#50 - ws_ext_discount_amt#49)))] + +(55) CometFilter +Input [2]: [customer_id#57, year_total#58] +Condition : (isnotnull(year_total#58) AND (year_total#58 > 0.00)) + +(56) ColumnarToRow [codegen id : 11] +Input [2]: [customer_id#57, year_total#58] + +(57) BroadcastExchange +Input [2]: [customer_id#57, year_total#58] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(52) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#59] +(58) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#57] Join type: Inner Join condition: None -(53) Project [codegen id : 16] -Output [5]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60] -Input [6]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, customer_id#59, year_total#60] +(59) Project [codegen id : 16] +Output [5]: [customer_id#18, year_total#19, customer_preferred_cust_flag#38, year_total#39, year_total#58] +Input [6]: [customer_id#18, year_total#19, customer_preferred_cust_flag#38, year_total#39, customer_id#57, year_total#58] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] +Output [8]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(55) CometFilter -Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] -Condition : (isnotnull(c_customer_sk#61) AND isnotnull(c_customer_id#62)) +(61) CometFilter +Input [8]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66] +Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) -(56) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] +(62) ColumnarToRow [codegen id : 14] +Input [8]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Output [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ws_sold_date_sk#70), dynamicpruningexpression(ws_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(58) CometFilter -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_bill_customer_sk#69) +(64) CometFilter +Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +Condition : isnotnull(ws_bill_customer_sk#67) -(59) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +(65) ColumnarToRow [codegen id : 12] +Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -(60) BroadcastExchange -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +(66) BroadcastExchange +Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#61] -Right keys [1]: [ws_bill_customer_sk#69] +(67) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#59] +Right keys [1]: [ws_bill_customer_sk#67] Join type: Inner Join condition: None -(62) Project [codegen id : 14] -Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -Input [12]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +(68) Project [codegen id : 14] +Output [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +Input [12]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -(63) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#74, d_year#75] +(69) ReusedExchange [Reuses operator id: 88] +Output [2]: [d_date_sk#72, d_year#73] -(64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#74] +(70) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#70] +Right keys [1]: [d_date_sk#72] Join type: Inner Join condition: None -(65) Project [codegen id : 14] -Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] -Input [12]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72, d_date_sk#74, d_year#75] - -(66) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] -Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] -Aggregate Attributes [1]: [sum#76] -Results [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] - -(67) Exchange -Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] -Arguments: hashpartitioning(c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(68) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] -Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#58] -Results [2]: [c_customer_id#62 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#58,18,2) AS year_total#79] - -(69) BroadcastExchange -Input [2]: [customer_id#78, year_total#79] +(71) Project [codegen id : 14] +Output [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#73] +Input [12]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70, d_date_sk#72, d_year#73] + +(72) HashAggregate [codegen id : 14] +Input [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#73] +Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] +Aggregate Attributes [1]: [sum#74] +Results [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73, sum#75] + +(73) RowToColumnar +Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73, sum#75] + +(74) CometColumnarExchange +Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73, sum#75] +Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(75) CometHashAggregate +Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73, sum#75] +Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] + +(76) ColumnarToRow [codegen id : 15] +Input [2]: [customer_id#76, year_total#77] + +(77) BroadcastExchange +Input [2]: [customer_id#76, year_total#77] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(70) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#78] +(78) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#76] Join type: Inner -Join condition: (CASE WHEN (year_total#60 > 0.00) THEN (year_total#79 / year_total#60) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END) +Join condition: (CASE WHEN (year_total#58 > 0.00) THEN (year_total#77 / year_total#58) END > CASE WHEN (year_total#19 > 0.00) THEN (year_total#39 / year_total#19) END) -(71) Project [codegen id : 16] -Output [1]: [customer_preferred_cust_flag#39] -Input [7]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60, customer_id#78, year_total#79] +(79) Project [codegen id : 16] +Output [1]: [customer_preferred_cust_flag#38] +Input [7]: [customer_id#18, year_total#19, customer_preferred_cust_flag#38, year_total#39, year_total#58, customer_id#76, year_total#77] -(72) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#39] -Arguments: 100, [customer_preferred_cust_flag#39 ASC NULLS FIRST], [customer_preferred_cust_flag#39] +(80) TakeOrderedAndProject +Input [1]: [customer_preferred_cust_flag#38] +Arguments: 100, [customer_preferred_cust_flag#38 ASC NULLS FIRST], [customer_preferred_cust_flag#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +BroadcastExchange (84) ++- * ColumnarToRow (83) + +- CometFilter (82) + +- CometScan parquet spark_catalog.default.date_dim (81) (unknown) Scan parquet spark_catalog.default.date_dim @@ -439,44 +463,44 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(74) CometFilter +(82) CometFilter Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(75) ColumnarToRow [codegen id : 1] +(83) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(76) BroadcastExchange +(84) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 -BroadcastExchange (80) -+- * ColumnarToRow (79) - +- CometFilter (78) - +- CometScan parquet spark_catalog.default.date_dim (77) +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 +BroadcastExchange (88) ++- * ColumnarToRow (87) + +- CometFilter (86) + +- CometScan parquet spark_catalog.default.date_dim (85) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#34, d_year#35] +Output [2]: [d_date_sk#33, d_year#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(78) CometFilter -Input [2]: [d_date_sk#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) +(86) CometFilter +Input [2]: [d_date_sk#33, d_year#34] +Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) -(79) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_year#35] +(87) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#33, d_year#34] -(80) BroadcastExchange -Input [2]: [d_date_sk#34, d_year#35] +(88) BroadcastExchange +Input [2]: [d_date_sk#33, d_year#34] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#33 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#70 IN dynamicpruning#32 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index 562b5fdf29..2810bbd068 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -6,43 +6,13 @@ TakeOrderedAndProject [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id] Project [customer_id,year_total,customer_preferred_cust_flag,year_total] BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [year_total] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + RowToColumnar + WholeStageCodegen (3) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -53,71 +23,109 @@ TakeOrderedAndProject [customer_preferred_cust_flag] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + ColumnarToRow + InputAdapter + CometFilter [year_total] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + ColumnarToRow InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 23ff8f948b..b59f6739bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.web_sales @@ -64,7 +66,7 @@ Join condition: None Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] +(10) ReusedExchange [Reuses operator id: 27] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -84,66 +86,70 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(14) RowToColumnar Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(15) CometColumnarExchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] -Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] -(16) Exchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(17) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(18) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(17) Sort [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [i_class#9 ASC NULLS FIRST], false, 0 +(19) ColumnarToRow [codegen id : 4] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] -(18) Window -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(20) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(19) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] -Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] +(21) Project [codegen id : 5] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] -(20) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +(22) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +Output [2]: [d_date_sk#11, d_date#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#18] +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#18] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index fae1c6dba1..985a824e6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -1,16 +1,16 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (5) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - Sort [i_class] + WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometSort [i_class] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt index 759871556e..a6d0b9b416 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt @@ -1,38 +1,40 @@ == Physical Plan == -* HashAggregate (34) -+- Exchange (33) - +- * HashAggregate (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store (4) - : : : +- BroadcastExchange (14) - : : : +- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (17) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.customer_demographics (20) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.household_demographics (26) +* ColumnarToRow (36) ++- CometHashAggregate (35) + +- CometColumnarExchange (34) + +- RowToColumnar (33) + +- * HashAggregate (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store (4) + : : : +- BroadcastExchange (14) + : : : +- * ColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.customer_address (10) + : : +- ReusedExchange (17) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.customer_demographics (20) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.household_demographics (26) (unknown) Scan parquet spark_catalog.default.store_sales @@ -110,7 +112,7 @@ Join condition: ((((ca_state#14 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#14] -(17) ReusedExchange [Reuses operator id: 39] +(17) ReusedExchange [Reuses operator id: 41] Output [1]: [d_date_sk#16] (18) BroadcastHashJoin [codegen id : 6] @@ -186,46 +188,50 @@ Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sal Aggregate Attributes [7]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28] Results [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] -(33) Exchange +(33) RowToColumnar Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(34) HashAggregate [codegen id : 7] +(34) CometColumnarExchange +Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(35) CometHashAggregate Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [4]: [avg(ss_quantity#5)#36, avg(UnscaledValue(ss_ext_sales_price#7))#37, avg(UnscaledValue(ss_ext_wholesale_cost#8))#38, sum(UnscaledValue(ss_ext_wholesale_cost#8))#39] -Results [4]: [avg(ss_quantity#5)#36 AS avg(ss_quantity)#40, cast((avg(UnscaledValue(ss_ext_sales_price#7))#37 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#41, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#38 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#42, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#39,17,2) AS sum(ss_ext_wholesale_cost)#43] + +(36) ColumnarToRow [codegen id : 7] +Input [4]: [avg(ss_quantity)#36, avg(ss_ext_sales_price)#37, avg(ss_ext_wholesale_cost)#38, sum(ss_ext_wholesale_cost)#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (39) -+- * ColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan parquet spark_catalog.default.date_dim (35) +BroadcastExchange (41) ++- * ColumnarToRow (40) + +- CometProject (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.date_dim (37) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_year#44] +Output [2]: [d_date_sk#16, d_year#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(36) CometFilter -Input [2]: [d_date_sk#16, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2001)) AND isnotnull(d_date_sk#16)) +(38) CometFilter +Input [2]: [d_date_sk#16, d_year#40] +Condition : ((isnotnull(d_year#40) AND (d_year#40 = 2001)) AND isnotnull(d_date_sk#16)) -(37) CometProject -Input [2]: [d_date_sk#16, d_year#44] +(39) CometProject +Input [2]: [d_date_sk#16, d_year#40] Arguments: [d_date_sk#16], [d_date_sk#16] -(38) ColumnarToRow [codegen id : 1] +(40) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] -(39) BroadcastExchange +(41) BroadcastExchange Input [1]: [d_date_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index 5e5fc41f83..87b7bf67e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -1,59 +1,61 @@ WholeStageCodegen (7) - HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (6) - HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] - Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] - Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] - CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometHashAggregate [sum,count,sum,count,sum,count,sum] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] + Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #6 + WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometFilter [hd_demo_sk,hd_dep_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 1323cb8df5..bfb15cc6a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -1,109 +1,115 @@ == Physical Plan == -TakeOrderedAndProject (105) -+- * HashAggregate (104) - +- Exchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * Project (67) - : +- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * ColumnarToRow (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) - : : +- BroadcastExchange (57) - : : +- * BroadcastHashJoin LeftSemi BuildRight (56) - : : :- * ColumnarToRow (54) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - :- * Project (83) - : +- * Filter (82) - : +- * HashAggregate (81) - : +- Exchange (80) - : +- * HashAggregate (79) - : +- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * Project (75) - : : +- * BroadcastHashJoin Inner BuildRight (74) - : : :- * BroadcastHashJoin LeftSemi BuildRight (72) - : : : :- * ColumnarToRow (70) - : : : : +- CometFilter (69) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) - : : : +- ReusedExchange (71) - : : +- ReusedExchange (73) - : +- ReusedExchange (76) - +- * Project (99) - +- * Filter (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * BroadcastHashJoin LeftSemi BuildRight (88) - : : :- * ColumnarToRow (86) - : : : +- CometFilter (85) - : : : +- CometScan parquet spark_catalog.default.web_sales (84) - : : +- ReusedExchange (87) - : +- ReusedExchange (89) - +- ReusedExchange (92) +* ColumnarToRow (111) ++- CometTakeOrderedAndProject (110) + +- CometHashAggregate (109) + +- CometColumnarExchange (108) + +- CometHashAggregate (107) + +- CometExpand (106) + +- CometUnion (105) + :- CometProject (70) + : +- CometFilter (69) + : +- CometHashAggregate (68) + : +- CometColumnarExchange (67) + : +- RowToColumnar (66) + : +- * HashAggregate (65) + : +- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * BroadcastHashJoin LeftSemi BuildRight (53) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (52) + : : : +- * Project (51) + : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : :- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (49) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) + : : : :- * ColumnarToRow (37) + : : : : +- CometHashAggregate (36) + : : : : +- CometColumnarExchange (35) + : : : : +- RowToColumnar (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * ColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (47) + : : : +- * Project (46) + : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : :- * Project (43) + : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : :- * ColumnarToRow (40) + : : : : : +- CometFilter (39) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) + : : : : +- ReusedExchange (41) + : : : +- ReusedExchange (44) + : : +- BroadcastExchange (59) + : : +- * BroadcastHashJoin LeftSemi BuildRight (58) + : : :- * ColumnarToRow (56) + : : : +- CometFilter (55) + : : : +- CometScan parquet spark_catalog.default.item (54) + : : +- ReusedExchange (57) + : +- ReusedExchange (62) + :- CometProject (87) + : +- CometFilter (86) + : +- CometHashAggregate (85) + : +- CometColumnarExchange (84) + : +- RowToColumnar (83) + : +- * HashAggregate (82) + : +- * Project (81) + : +- * BroadcastHashJoin Inner BuildRight (80) + : :- * Project (78) + : : +- * BroadcastHashJoin Inner BuildRight (77) + : : :- * BroadcastHashJoin LeftSemi BuildRight (75) + : : : :- * ColumnarToRow (73) + : : : : +- CometFilter (72) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (71) + : : : +- ReusedExchange (74) + : : +- ReusedExchange (76) + : +- ReusedExchange (79) + +- CometProject (104) + +- CometFilter (103) + +- CometHashAggregate (102) + +- CometColumnarExchange (101) + +- RowToColumnar (100) + +- * HashAggregate (99) + +- * Project (98) + +- * BroadcastHashJoin Inner BuildRight (97) + :- * Project (95) + : +- * BroadcastHashJoin Inner BuildRight (94) + : :- * BroadcastHashJoin LeftSemi BuildRight (92) + : : :- * ColumnarToRow (90) + : : : +- CometFilter (89) + : : : +- CometScan parquet spark_catalog.default.web_sales (88) + : : +- ReusedExchange (91) + : +- ReusedExchange (93) + +- ReusedExchange (96) (unknown) Scan parquet spark_catalog.default.store_sales @@ -207,7 +213,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 134] +(22) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -244,7 +250,7 @@ Join condition: None Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 134] +(30) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 6] @@ -264,16 +270,20 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) Exchange +(34) RowToColumnar Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) HashAggregate [codegen id : 10] +(35) CometColumnarExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(36) CometHashAggregate Input [3]: [brand_id#26, class_id#27, category_id#28] Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(37) ColumnarToRow [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] (unknown) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] @@ -283,68 +293,68 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(39) CometFilter Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] Condition : isnotnull(ws_item_sk#29) -(38) ColumnarToRow [codegen id : 9] +(40) ColumnarToRow [codegen id : 9] Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -(39) ReusedExchange [Reuses operator id: 19] +(41) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(40) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_item_sk#29] Right keys [1]: [i_item_sk#32] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(43) Project [codegen id : 9] Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(42) ReusedExchange [Reuses operator id: 134] +(44) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#36] -(43) BroadcastHashJoin [codegen id : 9] +(45) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#30] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(44) Project [codegen id : 9] +(46) Project [codegen id : 9] Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] -(45) BroadcastExchange +(47) BroadcastExchange Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(46) BroadcastHashJoin [codegen id : 10] +(48) BroadcastHashJoin [codegen id : 10] Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] Join type: LeftSemi Join condition: None -(47) BroadcastExchange +(49) BroadcastExchange Input [3]: [brand_id#26, class_id#27, category_id#28] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(48) BroadcastHashJoin [codegen id : 11] +(50) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#26, class_id#27, category_id#28] Join type: Inner Join condition: None -(49) Project [codegen id : 11] +(51) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#37] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [ss_item_sk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 25] +(53) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi @@ -357,444 +367,455 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Condition : isnotnull(i_item_sk#38) -(54) ColumnarToRow [codegen id : 23] +(56) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(55) ReusedExchange [Reuses operator id: 50] +(57) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(56) BroadcastHashJoin [codegen id : 23] +(58) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#38] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(57) BroadcastExchange +(59) BroadcastExchange Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(58) BroadcastHashJoin [codegen id : 25] +(60) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(59) Project [codegen id : 25] +(61) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(60) ReusedExchange [Reuses operator id: 129] +(62) ReusedExchange [Reuses operator id: 137] Output [1]: [d_date_sk#42] -(61) BroadcastHashJoin [codegen id : 25] +(63) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#42] Join type: Inner Join condition: None -(62) Project [codegen id : 25] +(64) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(63) HashAggregate [codegen id : 25] +(65) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(64) Exchange +(66) RowToColumnar +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(67) CometColumnarExchange Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(65) HashAggregate [codegen id : 26] +(68) CometHashAggregate Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#51, count(1)#50 AS number_sales#52] -(66) Filter [codegen id : 26] -Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) +(69) CometFilter +Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#49, number_sales#50] +Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(67) Project [codegen id : 26] -Output [6]: [sales#51, number_sales#52, store AS channel#55, i_brand_id#39 AS i_brand_id#56, i_class_id#40 AS i_class_id#57, i_category_id#41 AS i_category_id#58] -Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] +(70) CometProject +Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#49, number_sales#50] +Arguments: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56], [sales#49, number_sales#50, store AS channel#53, i_brand_id#39 AS i_brand_id#54, i_class_id#40 AS i_class_id#55, i_category_id#41 AS i_category_id#56] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#62), dynamicpruningexpression(cs_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) CometFilter -Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] -Condition : isnotnull(cs_item_sk#59) +(72) CometFilter +Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +Condition : isnotnull(cs_item_sk#57) -(70) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +(73) ColumnarToRow [codegen id : 50] +Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -(71) ReusedExchange [Reuses operator id: 50] +(74) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(72) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#59] +(75) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [cs_item_sk#57] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(73) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +(76) ReusedExchange [Reuses operator id: 59] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(74) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#59] -Right keys [1]: [i_item_sk#64] +(77) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [cs_item_sk#57] +Right keys [1]: [i_item_sk#62] Join type: Inner Join condition: None -(75) Project [codegen id : 51] -Output [6]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [8]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +(78) Project [codegen id : 50] +Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(76) ReusedExchange [Reuses operator id: 129] -Output [1]: [d_date_sk#68] +(79) ReusedExchange [Reuses operator id: 137] +Output [1]: [d_date_sk#66] -(77) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#62] -Right keys [1]: [d_date_sk#68] +(80) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [cs_sold_date_sk#60] +Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(78) Project [codegen id : 51] -Output [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [7]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] +(81) Project [codegen id : 50] +Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] + +(82) HashAggregate [codegen id : 50] +Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -(79) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [partial_sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), partial_count(1)] -Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +(83) RowToColumnar +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -(80) Exchange -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(84) CometColumnarExchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(81) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#75, count(1)#76] -Results [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#75 AS sales#77, count(1)#76 AS number_sales#78] +(85) CometHashAggregate +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] -(82) Filter [codegen id : 52] -Input [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#77, number_sales#78] -Condition : (isnotnull(sales#77) AND (cast(sales#77 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) +(86) CometFilter +Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#73, number_sales#74] +Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(83) Project [codegen id : 52] -Output [6]: [sales#77, number_sales#78, catalog AS channel#79, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#77, number_sales#78] +(87) CometProject +Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#73, number_sales#74] +Arguments: [sales#73, number_sales#74, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65], [sales#73, number_sales#74, catalog AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] +Output [4]: [ws_item_sk#76, ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#80)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(85) CometFilter -Input [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] -Condition : isnotnull(ws_item_sk#80) +(89) CometFilter +Input [4]: [ws_item_sk#76, ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_item_sk#76) -(86) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] +(90) ColumnarToRow [codegen id : 75] +Input [4]: [ws_item_sk#76, ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79] -(87) ReusedExchange [Reuses operator id: 50] +(91) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(88) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#80] +(92) BroadcastHashJoin [codegen id : 75] +Left keys [1]: [ws_item_sk#76] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(89) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] +(93) ReusedExchange [Reuses operator id: 59] +Output [4]: [i_item_sk#81, i_brand_id#82, i_class_id#83, i_category_id#84] -(90) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#80] -Right keys [1]: [i_item_sk#85] +(94) BroadcastHashJoin [codegen id : 75] +Left keys [1]: [ws_item_sk#76] +Right keys [1]: [i_item_sk#81] Join type: Inner Join condition: None -(91) Project [codegen id : 77] -Output [6]: [ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88] -Input [8]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] +(95) Project [codegen id : 75] +Output [6]: [ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79, i_brand_id#82, i_class_id#83, i_category_id#84] +Input [8]: [ws_item_sk#76, ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79, i_item_sk#81, i_brand_id#82, i_class_id#83, i_category_id#84] -(92) ReusedExchange [Reuses operator id: 129] -Output [1]: [d_date_sk#89] +(96) ReusedExchange [Reuses operator id: 137] +Output [1]: [d_date_sk#85] -(93) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#83] -Right keys [1]: [d_date_sk#89] +(97) BroadcastHashJoin [codegen id : 75] +Left keys [1]: [ws_sold_date_sk#79] +Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None -(94) Project [codegen id : 77] -Output [5]: [ws_quantity#81, ws_list_price#82, i_brand_id#86, i_class_id#87, i_category_id#88] -Input [7]: [ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, d_date_sk#89] - -(95) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#81, ws_list_price#82, i_brand_id#86, i_class_id#87, i_category_id#88] -Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] -Functions [2]: [partial_sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82)), partial_count(1)] -Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] -Results [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] - -(96) Exchange -Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, 5), ENSURE_REQUIREMENTS, [plan_id=11] - -(97) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] -Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] -Functions [2]: [sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82))#96, count(1)#97] -Results [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82))#96 AS sales#98, count(1)#97 AS number_sales#99] - -(98) Filter [codegen id : 78] -Input [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sales#98, number_sales#99] -Condition : (isnotnull(sales#98) AND (cast(sales#98 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(99) Project [codegen id : 78] -Output [6]: [sales#98, number_sales#99, web AS channel#100, i_brand_id#86, i_class_id#87, i_category_id#88] -Input [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sales#98, number_sales#99] - -(100) Union - -(101) Expand [codegen id : 79] -Input [6]: [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58] -Arguments: [[sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58, 0], [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, null, 1], [sales#51, number_sales#52, channel#55, i_brand_id#56, null, null, 3], [sales#51, number_sales#52, channel#55, null, null, null, 7], [sales#51, number_sales#52, null, null, null, null, 15]], [sales#51, number_sales#52, channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] - -(102) HashAggregate [codegen id : 79] -Input [7]: [sales#51, number_sales#52, channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] -Keys [5]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] -Functions [2]: [partial_sum(sales#51), partial_sum(number_sales#52)] -Aggregate Attributes [3]: [sum#106, isEmpty#107, sum#108] -Results [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] - -(103) Exchange -Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] -Arguments: hashpartitioning(channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(104) HashAggregate [codegen id : 80] -Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] -Keys [5]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] -Functions [2]: [sum(sales#51), sum(number_sales#52)] -Aggregate Attributes [2]: [sum(sales#51)#112, sum(number_sales#52)#113] -Results [6]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales#51)#112 AS sum(sales)#114, sum(number_sales#52)#113 AS sum(number_sales)#115] - -(105) TakeOrderedAndProject -Input [6]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales)#114, sum(number_sales)#115] -Arguments: 100, [channel#101 ASC NULLS FIRST, i_brand_id#102 ASC NULLS FIRST, i_class_id#103 ASC NULLS FIRST, i_category_id#104 ASC NULLS FIRST], [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales)#114, sum(number_sales)#115] +(98) Project [codegen id : 75] +Output [5]: [ws_quantity#77, ws_list_price#78, i_brand_id#82, i_class_id#83, i_category_id#84] +Input [7]: [ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79, i_brand_id#82, i_class_id#83, i_category_id#84, d_date_sk#85] + +(99) HashAggregate [codegen id : 75] +Input [5]: [ws_quantity#77, ws_list_price#78, i_brand_id#82, i_class_id#83, i_category_id#84] +Keys [3]: [i_brand_id#82, i_class_id#83, i_category_id#84] +Functions [2]: [partial_sum((cast(ws_quantity#77 as decimal(10,0)) * ws_list_price#78)), partial_count(1)] +Aggregate Attributes [3]: [sum#86, isEmpty#87, count#88] +Results [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] + +(100) RowToColumnar +Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] + +(101) CometColumnarExchange +Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] +Arguments: hashpartitioning(i_brand_id#82, i_class_id#83, i_category_id#84, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(102) CometHashAggregate +Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] +Keys [3]: [i_brand_id#82, i_class_id#83, i_category_id#84] +Functions [2]: [sum((cast(ws_quantity#77 as decimal(10,0)) * ws_list_price#78)), count(1)] + +(103) CometFilter +Input [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sales#92, number_sales#93] +Condition : (isnotnull(sales#92) AND (cast(sales#92 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) + +(104) CometProject +Input [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sales#92, number_sales#93] +Arguments: [sales#92, number_sales#93, channel#94, i_brand_id#82, i_class_id#83, i_category_id#84], [sales#92, number_sales#93, web AS channel#94, i_brand_id#82, i_class_id#83, i_category_id#84] + +(105) CometUnion +Child 0 Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Child 1 Input [6]: [sales#73, number_sales#74, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65] +Child 2 Input [6]: [sales#92, number_sales#93, channel#94, i_brand_id#82, i_class_id#83, i_category_id#84] + +(106) CometExpand +Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99] + +(107) CometHashAggregate +Input [7]: [sales#49, number_sales#50, channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99] +Keys [5]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99] +Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] + +(108) CometColumnarExchange +Input [8]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99, sum#100, isEmpty#101, sum#102] +Arguments: hashpartitioning(channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(109) CometHashAggregate +Input [8]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99, sum#100, isEmpty#101, sum#102] +Keys [5]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99] +Functions [2]: [sum(sales#49), sum(number_sales#50)] + +(110) CometTakeOrderedAndProject +Input [6]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, sum(sales)#103, sum(number_sales)#104] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#95 ASC NULLS FIRST,i_brand_id#96 ASC NULLS FIRST,i_class_id#97 ASC NULLS FIRST,i_category_id#98 ASC NULLS FIRST], output=[channel#95,i_brand_id#96,i_class_id#97,i_category_id#98,sum(sales)#103,sum(number_sales)#104]), 100, [channel#95 ASC NULLS FIRST, i_brand_id#96 ASC NULLS FIRST, i_class_id#97 ASC NULLS FIRST, i_category_id#98 ASC NULLS FIRST], [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, sum(sales)#103, sum(number_sales)#104] + +(111) ColumnarToRow [codegen id : 76] +Input [6]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, sum(sales)#103, sum(number_sales)#104] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* HashAggregate (124) -+- Exchange (123) - +- * HashAggregate (122) - +- Union (121) - :- * Project (110) - : +- * BroadcastHashJoin Inner BuildRight (109) - : :- * ColumnarToRow (107) - : : +- CometScan parquet spark_catalog.default.store_sales (106) - : +- ReusedExchange (108) - :- * Project (115) - : +- * BroadcastHashJoin Inner BuildRight (114) - : :- * ColumnarToRow (112) - : : +- CometScan parquet spark_catalog.default.catalog_sales (111) - : +- ReusedExchange (113) - +- * Project (120) - +- * BroadcastHashJoin Inner BuildRight (119) - :- * ColumnarToRow (117) - : +- CometScan parquet spark_catalog.default.web_sales (116) - +- ReusedExchange (118) +Subquery:1 Hosting operator id = 69 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +* ColumnarToRow (132) ++- CometHashAggregate (131) + +- CometColumnarExchange (130) + +- RowToColumnar (129) + +- * HashAggregate (128) + +- Union (127) + :- * Project (116) + : +- * BroadcastHashJoin Inner BuildRight (115) + : :- * ColumnarToRow (113) + : : +- CometScan parquet spark_catalog.default.store_sales (112) + : +- ReusedExchange (114) + :- * Project (121) + : +- * BroadcastHashJoin Inner BuildRight (120) + : :- * ColumnarToRow (118) + : : +- CometScan parquet spark_catalog.default.catalog_sales (117) + : +- ReusedExchange (119) + +- * Project (126) + +- * BroadcastHashJoin Inner BuildRight (125) + :- * ColumnarToRow (123) + : +- CometScan parquet spark_catalog.default.web_sales (122) + +- ReusedExchange (124) (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#116, ss_list_price#117, ss_sold_date_sk#118] +Output [3]: [ss_quantity#105, ss_list_price#106, ss_sold_date_sk#107] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#118), dynamicpruningexpression(ss_sold_date_sk#118 IN dynamicpruning#119)] +PartitionFilters: [isnotnull(ss_sold_date_sk#107), dynamicpruningexpression(ss_sold_date_sk#107 IN dynamicpruning#108)] ReadSchema: struct -(107) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#116, ss_list_price#117, ss_sold_date_sk#118] +(113) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#105, ss_list_price#106, ss_sold_date_sk#107] -(108) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#120] +(114) ReusedExchange [Reuses operator id: 142] +Output [1]: [d_date_sk#109] -(109) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#118] -Right keys [1]: [d_date_sk#120] +(115) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#107] +Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(110) Project [codegen id : 2] -Output [2]: [ss_quantity#116 AS quantity#121, ss_list_price#117 AS list_price#122] -Input [4]: [ss_quantity#116, ss_list_price#117, ss_sold_date_sk#118, d_date_sk#120] +(116) Project [codegen id : 2] +Output [2]: [ss_quantity#105 AS quantity#110, ss_list_price#106 AS list_price#111] +Input [4]: [ss_quantity#105, ss_list_price#106, ss_sold_date_sk#107, d_date_sk#109] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#123, cs_list_price#124, cs_sold_date_sk#125] +Output [3]: [cs_quantity#112, cs_list_price#113, cs_sold_date_sk#114] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#125), dynamicpruningexpression(cs_sold_date_sk#125 IN dynamicpruning#126)] +PartitionFilters: [isnotnull(cs_sold_date_sk#114), dynamicpruningexpression(cs_sold_date_sk#114 IN dynamicpruning#115)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#123, cs_list_price#124, cs_sold_date_sk#125] +(118) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#112, cs_list_price#113, cs_sold_date_sk#114] -(113) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#127] +(119) ReusedExchange [Reuses operator id: 142] +Output [1]: [d_date_sk#116] -(114) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#125] -Right keys [1]: [d_date_sk#127] +(120) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#114] +Right keys [1]: [d_date_sk#116] Join type: Inner Join condition: None -(115) Project [codegen id : 4] -Output [2]: [cs_quantity#123 AS quantity#128, cs_list_price#124 AS list_price#129] -Input [4]: [cs_quantity#123, cs_list_price#124, cs_sold_date_sk#125, d_date_sk#127] +(121) Project [codegen id : 4] +Output [2]: [cs_quantity#112 AS quantity#117, cs_list_price#113 AS list_price#118] +Input [4]: [cs_quantity#112, cs_list_price#113, cs_sold_date_sk#114, d_date_sk#116] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#130, ws_list_price#131, ws_sold_date_sk#132] +Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#132), dynamicpruningexpression(ws_sold_date_sk#132 IN dynamicpruning#133)] +PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#122)] ReadSchema: struct -(117) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#130, ws_list_price#131, ws_sold_date_sk#132] +(123) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] -(118) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#134] +(124) ReusedExchange [Reuses operator id: 142] +Output [1]: [d_date_sk#123] -(119) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#132] -Right keys [1]: [d_date_sk#134] +(125) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#121] +Right keys [1]: [d_date_sk#123] Join type: Inner Join condition: None -(120) Project [codegen id : 6] -Output [2]: [ws_quantity#130 AS quantity#135, ws_list_price#131 AS list_price#136] -Input [4]: [ws_quantity#130, ws_list_price#131, ws_sold_date_sk#132, d_date_sk#134] +(126) Project [codegen id : 6] +Output [2]: [ws_quantity#119 AS quantity#124, ws_list_price#120 AS list_price#125] +Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#123] -(121) Union +(127) Union -(122) HashAggregate [codegen id : 7] -Input [2]: [quantity#121, list_price#122] +(128) HashAggregate [codegen id : 7] +Input [2]: [quantity#110, list_price#111] Keys: [] -Functions [1]: [partial_avg((cast(quantity#121 as decimal(10,0)) * list_price#122))] -Aggregate Attributes [2]: [sum#137, count#138] -Results [2]: [sum#139, count#140] +Functions [1]: [partial_avg((cast(quantity#110 as decimal(10,0)) * list_price#111))] +Aggregate Attributes [2]: [sum#126, count#127] +Results [2]: [sum#128, count#129] + +(129) RowToColumnar +Input [2]: [sum#128, count#129] -(123) Exchange -Input [2]: [sum#139, count#140] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(130) CometColumnarExchange +Input [2]: [sum#128, count#129] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(124) HashAggregate [codegen id : 8] -Input [2]: [sum#139, count#140] +(131) CometHashAggregate +Input [2]: [sum#128, count#129] Keys: [] -Functions [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))] -Aggregate Attributes [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))#141] -Results [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))#141 AS average_sales#142] +Functions [1]: [avg((cast(quantity#110 as decimal(10,0)) * list_price#111))] + +(132) ColumnarToRow [codegen id : 8] +Input [1]: [average_sales#130] -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#118 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#107 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 111 Hosting Expression = cs_sold_date_sk#125 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 117 Hosting Expression = cs_sold_date_sk#114 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 116 Hosting Expression = ws_sold_date_sk#132 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 122 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (129) -+- * ColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan parquet spark_catalog.default.date_dim (125) +BroadcastExchange (137) ++- * ColumnarToRow (136) + +- CometProject (135) + +- CometFilter (134) + +- CometScan parquet spark_catalog.default.date_dim (133) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#42, d_year#143, d_moy#144] +Output [3]: [d_date_sk#42, d_year#131, d_moy#132] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(126) CometFilter -Input [3]: [d_date_sk#42, d_year#143, d_moy#144] -Condition : ((((isnotnull(d_year#143) AND isnotnull(d_moy#144)) AND (d_year#143 = 2001)) AND (d_moy#144 = 11)) AND isnotnull(d_date_sk#42)) +(134) CometFilter +Input [3]: [d_date_sk#42, d_year#131, d_moy#132] +Condition : ((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND (d_year#131 = 2001)) AND (d_moy#132 = 11)) AND isnotnull(d_date_sk#42)) -(127) CometProject -Input [3]: [d_date_sk#42, d_year#143, d_moy#144] +(135) CometProject +Input [3]: [d_date_sk#42, d_year#131, d_moy#132] Arguments: [d_date_sk#42], [d_date_sk#42] -(128) ColumnarToRow [codegen id : 1] +(136) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#42] -(129) BroadcastExchange +(137) BroadcastExchange Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (134) -+- * ColumnarToRow (133) - +- CometProject (132) - +- CometFilter (131) - +- CometScan parquet spark_catalog.default.date_dim (130) +BroadcastExchange (142) ++- * ColumnarToRow (141) + +- CometProject (140) + +- CometFilter (139) + +- CometScan parquet spark_catalog.default.date_dim (138) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#145] +Output [2]: [d_date_sk#25, d_year#133] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(131) CometFilter -Input [2]: [d_date_sk#25, d_year#145] -Condition : (((isnotnull(d_year#145) AND (d_year#145 >= 1999)) AND (d_year#145 <= 2001)) AND isnotnull(d_date_sk#25)) +(139) CometFilter +Input [2]: [d_date_sk#25, d_year#133] +Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1999)) AND (d_year#133 <= 2001)) AND isnotnull(d_date_sk#25)) -(132) CometProject -Input [2]: [d_date_sk#25, d_year#145] +(140) CometProject +Input [2]: [d_date_sk#25, d_year#133] Arguments: [d_date_sk#25], [d_date_sk#25] -(133) ColumnarToRow [codegen id : 1] +(141) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(134) BroadcastExchange +(142) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:9 Hosting operator id = 86 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 71 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:11 Hosting operator id = 103 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index cf688c4486..cd494e1546 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -1,21 +1,21 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] - InputAdapter - Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - InputAdapter - Union - WholeStageCodegen (26) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #13 +WholeStageCodegen (76) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] + CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] + CometUnion + CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + CometFilter [sales] + Subquery #3 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometHashAggregate [sum,count] + CometColumnarExchange #13 + RowToColumnar WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -47,44 +47,46 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #7 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 + RowToColumnar + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ColumnarToRow + InputAdapter + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #6 + RowToColumnar WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] @@ -134,81 +136,79 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [d_date_sk] #7 InputAdapter ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (52) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow + BroadcastExchange #12 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #3 + CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + CometFilter [sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 + RowToColumnar + WholeStageCodegen (50) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (78) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + InputAdapter + ReusedExchange [d_date_sk] #3 + CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + CometFilter [sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #15 + RowToColumnar + WholeStageCodegen (75) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 536306de5d..3ce08e1136 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -1,88 +1,94 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * BroadcastHashJoin Inner BuildRight (83) - :- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * ColumnarToRow (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) - : : +- BroadcastExchange (57) - : : +- * BroadcastHashJoin LeftSemi BuildRight (56) - : : :- * ColumnarToRow (54) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - +- BroadcastExchange (82) - +- * Filter (81) - +- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * BroadcastHashJoin Inner BuildRight (76) - :- * Project (74) - : +- * BroadcastHashJoin Inner BuildRight (73) - : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : :- * ColumnarToRow (69) - : : : +- CometFilter (68) - : : : +- CometScan parquet spark_catalog.default.store_sales (67) - : : +- ReusedExchange (70) - : +- ReusedExchange (72) - +- ReusedExchange (75) +TakeOrderedAndProject (90) ++- * BroadcastHashJoin Inner BuildRight (89) + :- * ColumnarToRow (70) + : +- CometFilter (69) + : +- CometHashAggregate (68) + : +- CometColumnarExchange (67) + : +- RowToColumnar (66) + : +- * HashAggregate (65) + : +- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * BroadcastHashJoin LeftSemi BuildRight (53) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (52) + : : : +- * Project (51) + : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : :- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (49) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) + : : : :- * ColumnarToRow (37) + : : : : +- CometHashAggregate (36) + : : : : +- CometColumnarExchange (35) + : : : : +- RowToColumnar (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * ColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (47) + : : : +- * Project (46) + : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : :- * Project (43) + : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : :- * ColumnarToRow (40) + : : : : : +- CometFilter (39) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) + : : : : +- ReusedExchange (41) + : : : +- ReusedExchange (44) + : : +- BroadcastExchange (59) + : : +- * BroadcastHashJoin LeftSemi BuildRight (58) + : : :- * ColumnarToRow (56) + : : : +- CometFilter (55) + : : : +- CometScan parquet spark_catalog.default.item (54) + : : +- ReusedExchange (57) + : +- ReusedExchange (62) + +- BroadcastExchange (88) + +- * ColumnarToRow (87) + +- CometFilter (86) + +- CometHashAggregate (85) + +- CometColumnarExchange (84) + +- RowToColumnar (83) + +- * HashAggregate (82) + +- * Project (81) + +- * BroadcastHashJoin Inner BuildRight (80) + :- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * BroadcastHashJoin LeftSemi BuildRight (75) + : : :- * ColumnarToRow (73) + : : : +- CometFilter (72) + : : : +- CometScan parquet spark_catalog.default.store_sales (71) + : : +- ReusedExchange (74) + : +- ReusedExchange (76) + +- ReusedExchange (79) (unknown) Scan parquet spark_catalog.default.store_sales @@ -186,7 +192,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 117] +(22) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -223,7 +229,7 @@ Join condition: None Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 117] +(30) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 6] @@ -243,16 +249,20 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) Exchange +(34) RowToColumnar Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) HashAggregate [codegen id : 10] +(35) CometColumnarExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(36) CometHashAggregate Input [3]: [brand_id#26, class_id#27, category_id#28] Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(37) ColumnarToRow [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] (unknown) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] @@ -262,68 +272,68 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(39) CometFilter Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] Condition : isnotnull(ws_item_sk#29) -(38) ColumnarToRow [codegen id : 9] +(40) ColumnarToRow [codegen id : 9] Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -(39) ReusedExchange [Reuses operator id: 19] +(41) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(40) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_item_sk#29] Right keys [1]: [i_item_sk#32] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(43) Project [codegen id : 9] Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(42) ReusedExchange [Reuses operator id: 117] +(44) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#36] -(43) BroadcastHashJoin [codegen id : 9] +(45) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#30] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(44) Project [codegen id : 9] +(46) Project [codegen id : 9] Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] -(45) BroadcastExchange +(47) BroadcastExchange Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(46) BroadcastHashJoin [codegen id : 10] +(48) BroadcastHashJoin [codegen id : 10] Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] Join type: LeftSemi Join condition: None -(47) BroadcastExchange +(49) BroadcastExchange Input [3]: [brand_id#26, class_id#27, category_id#28] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(48) BroadcastHashJoin [codegen id : 11] +(50) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#26, class_id#27, category_id#28] Join type: Inner Join condition: None -(49) Project [codegen id : 11] +(51) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#37] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [ss_item_sk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 25] +(53) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi @@ -336,420 +346,434 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) -(54) ColumnarToRow [codegen id : 23] +(56) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(55) ReusedExchange [Reuses operator id: 50] +(57) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(56) BroadcastHashJoin [codegen id : 23] +(58) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#38] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(57) BroadcastExchange +(59) BroadcastExchange Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(58) BroadcastHashJoin [codegen id : 25] +(60) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(59) Project [codegen id : 25] +(61) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(60) ReusedExchange [Reuses operator id: 108] +(62) ReusedExchange [Reuses operator id: 116] Output [1]: [d_date_sk#42] -(61) BroadcastHashJoin [codegen id : 25] +(63) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#42] Join type: Inner Join condition: None -(62) Project [codegen id : 25] +(64) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(63) HashAggregate [codegen id : 25] +(65) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(64) Exchange +(66) RowToColumnar +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(67) CometColumnarExchange Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(65) HashAggregate [codegen id : 52] +(68) CometHashAggregate Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(66) Filter [codegen id : 52] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(69) CometFilter +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] +Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(70) ColumnarToRow [codegen id : 52] +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_sold_date_sk#57 IN dynamicpruning#58)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(68) CometFilter -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] -Condition : isnotnull(ss_item_sk#56) +(72) CometFilter +Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] +Condition : isnotnull(ss_item_sk#54) -(69) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +(73) ColumnarToRow [codegen id : 50] +Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(70) ReusedExchange [Reuses operator id: 50] +(74) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(71) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] +(75) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(72) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +(76) ReusedExchange [Reuses operator id: 59] +Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [i_item_sk#61] +(77) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#59] Join type: Inner Join condition: None -(74) Project [codegen id : 50] -Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +(78) Project [codegen id : 50] +Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] +Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(75) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#65] +(79) ReusedExchange [Reuses operator id: 130] +Output [1]: [d_date_sk#63] -(76) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#65] +(80) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_sold_date_sk#57] +Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(77) Project [codegen id : 50] -Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] - -(78) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] -Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] - -(79) Exchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(80) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72, count(1)#73] -Results [6]: [store AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] - -(81) Filter [codegen id : 51] -Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) - -(82) BroadcastExchange -Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] +(81) Project [codegen id : 50] +Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] +Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] + +(82) HashAggregate [codegen id : 50] +Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] +Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] +Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] + +(83) RowToColumnar +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] + +(84) CometColumnarExchange +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(85) CometHashAggregate +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] + +(86) CometFilter +Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +Condition : (isnotnull(sales#71) AND (cast(sales#71 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(87) ColumnarToRow [codegen id : 51] +Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] + +(88) BroadcastExchange +Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(83) BroadcastHashJoin [codegen id : 52] +(89) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Right keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Join type: Inner Join condition: None -(84) TakeOrderedAndProject -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] +(90) TakeOrderedAndProject +Input [12]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51, channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51, channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (103) -+- Exchange (102) - +- * HashAggregate (101) - +- Union (100) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * ColumnarToRow (86) - : : +- CometScan parquet spark_catalog.default.store_sales (85) - : +- ReusedExchange (87) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * ColumnarToRow (91) - : : +- CometScan parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (92) - +- * Project (99) - +- * BroadcastHashJoin Inner BuildRight (98) - :- * ColumnarToRow (96) - : +- CometScan parquet spark_catalog.default.web_sales (95) - +- ReusedExchange (97) +Subquery:1 Hosting operator id = 69 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* ColumnarToRow (111) ++- CometHashAggregate (110) + +- CometColumnarExchange (109) + +- RowToColumnar (108) + +- * HashAggregate (107) + +- Union (106) + :- * Project (95) + : +- * BroadcastHashJoin Inner BuildRight (94) + : :- * ColumnarToRow (92) + : : +- CometScan parquet spark_catalog.default.store_sales (91) + : +- ReusedExchange (93) + :- * Project (100) + : +- * BroadcastHashJoin Inner BuildRight (99) + : :- * ColumnarToRow (97) + : : +- CometScan parquet spark_catalog.default.catalog_sales (96) + : +- ReusedExchange (98) + +- * Project (105) + +- * BroadcastHashJoin Inner BuildRight (104) + :- * ColumnarToRow (102) + : +- CometScan parquet spark_catalog.default.web_sales (101) + +- ReusedExchange (103) (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] +Output [3]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#79), dynamicpruningexpression(ss_sold_date_sk#79 IN dynamicpruning#80)] +PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_sold_date_sk#75 IN dynamicpruning#76)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] +(92) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75] -(87) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#81] +(93) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#77] -(88) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#79] -Right keys [1]: [d_date_sk#81] +(94) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#75] +Right keys [1]: [d_date_sk#77] Join type: Inner Join condition: None -(89) Project [codegen id : 2] -Output [2]: [ss_quantity#77 AS quantity#82, ss_list_price#78 AS list_price#83] -Input [4]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79, d_date_sk#81] +(95) Project [codegen id : 2] +Output [2]: [ss_quantity#73 AS quantity#78, ss_list_price#74 AS list_price#79] +Input [4]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75, d_date_sk#77] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] +Output [3]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#86), dynamicpruningexpression(cs_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(cs_sold_date_sk#82), dynamicpruningexpression(cs_sold_date_sk#82 IN dynamicpruning#83)] ReadSchema: struct -(91) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] +(97) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82] -(92) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#88] +(98) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#84] -(93) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#86] -Right keys [1]: [d_date_sk#88] +(99) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#82] +Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(94) Project [codegen id : 4] -Output [2]: [cs_quantity#84 AS quantity#89, cs_list_price#85 AS list_price#90] -Input [4]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86, d_date_sk#88] +(100) Project [codegen id : 4] +Output [2]: [cs_quantity#80 AS quantity#85, cs_list_price#81 AS list_price#86] +Input [4]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82, d_date_sk#84] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] +PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#90)] ReadSchema: struct -(96) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +(102) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] -(97) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#95] +(103) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#91] -(98) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#93] -Right keys [1]: [d_date_sk#95] +(104) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#89] +Right keys [1]: [d_date_sk#91] Join type: Inner Join condition: None -(99) Project [codegen id : 6] -Output [2]: [ws_quantity#91 AS quantity#96, ws_list_price#92 AS list_price#97] -Input [4]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93, d_date_sk#95] +(105) Project [codegen id : 6] +Output [2]: [ws_quantity#87 AS quantity#92, ws_list_price#88 AS list_price#93] +Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#91] -(100) Union +(106) Union -(101) HashAggregate [codegen id : 7] -Input [2]: [quantity#82, list_price#83] +(107) HashAggregate [codegen id : 7] +Input [2]: [quantity#78, list_price#79] Keys: [] -Functions [1]: [partial_avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] -Aggregate Attributes [2]: [sum#98, count#99] -Results [2]: [sum#100, count#101] +Functions [1]: [partial_avg((cast(quantity#78 as decimal(10,0)) * list_price#79))] +Aggregate Attributes [2]: [sum#94, count#95] +Results [2]: [sum#96, count#97] -(102) Exchange -Input [2]: [sum#100, count#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(108) RowToColumnar +Input [2]: [sum#96, count#97] -(103) HashAggregate [codegen id : 8] -Input [2]: [sum#100, count#101] +(109) CometColumnarExchange +Input [2]: [sum#96, count#97] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(110) CometHashAggregate +Input [2]: [sum#96, count#97] Keys: [] -Functions [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] -Aggregate Attributes [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102] -Results [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102 AS average_sales#103] +Functions [1]: [avg((cast(quantity#78 as decimal(10,0)) * list_price#79))] + +(111) ColumnarToRow [codegen id : 8] +Input [1]: [average_sales#98] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#79 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#75 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#82 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * ColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan parquet spark_catalog.default.date_dim (104) +BroadcastExchange (116) ++- * ColumnarToRow (115) + +- CometProject (114) + +- CometFilter (113) + +- CometScan parquet spark_catalog.default.date_dim (112) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#42, d_week_seq#104] +Output [2]: [d_date_sk#42, d_week_seq#99] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(105) CometFilter -Input [2]: [d_date_sk#42, d_week_seq#104] -Condition : ((isnotnull(d_week_seq#104) AND (d_week_seq#104 = Subquery scalar-subquery#105, [id=#106])) AND isnotnull(d_date_sk#42)) +(113) CometFilter +Input [2]: [d_date_sk#42, d_week_seq#99] +Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#42)) -(106) CometProject -Input [2]: [d_date_sk#42, d_week_seq#104] +(114) CometProject +Input [2]: [d_date_sk#42, d_week_seq#99] Arguments: [d_date_sk#42], [d_date_sk#42] -(107) ColumnarToRow [codegen id : 1] +(115) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#42] -(108) BroadcastExchange +(116) BroadcastExchange Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 105 Hosting Expression = Subquery scalar-subquery#105, [id=#106] -* ColumnarToRow (112) -+- CometProject (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +Subquery:6 Hosting operator id = 113 Hosting Expression = Subquery scalar-subquery#100, [id=#101] +* ColumnarToRow (120) ++- CometProject (119) + +- CometFilter (118) + +- CometScan parquet spark_catalog.default.date_dim (117) (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] +Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(110) CometFilter -Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] -Condition : (((((isnotnull(d_year#108) AND isnotnull(d_moy#109)) AND isnotnull(d_dom#110)) AND (d_year#108 = 2000)) AND (d_moy#109 = 12)) AND (d_dom#110 = 11)) +(118) CometFilter +Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] +Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 2000)) AND (d_moy#104 = 12)) AND (d_dom#105 = 11)) -(111) CometProject -Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] -Arguments: [d_week_seq#107], [d_week_seq#107] +(119) CometProject +Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] +Arguments: [d_week_seq#102], [d_week_seq#102] -(112) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#107] +(120) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#102] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (125) ++- * ColumnarToRow (124) + +- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#111] +Output [2]: [d_date_sk#25, d_year#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter -Input [2]: [d_date_sk#25, d_year#111] -Condition : (((isnotnull(d_year#111) AND (d_year#111 >= 1999)) AND (d_year#111 <= 2001)) AND isnotnull(d_date_sk#25)) +(122) CometFilter +Input [2]: [d_date_sk#25, d_year#106] +Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1999)) AND (d_year#106 <= 2001)) AND isnotnull(d_date_sk#25)) -(115) CometProject -Input [2]: [d_date_sk#25, d_year#111] +(123) CometProject +Input [2]: [d_date_sk#25, d_year#106] Arguments: [d_date_sk#25], [d_date_sk#25] -(116) ColumnarToRow [codegen id : 1] +(124) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(117) BroadcastExchange +(125) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:10 Hosting operator id = 86 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 -BroadcastExchange (122) -+- * ColumnarToRow (121) - +- CometProject (120) - +- CometFilter (119) - +- CometScan parquet spark_catalog.default.date_dim (118) +Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 +BroadcastExchange (130) ++- * ColumnarToRow (129) + +- CometProject (128) + +- CometFilter (127) + +- CometScan parquet spark_catalog.default.date_dim (126) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#112] +Output [2]: [d_date_sk#63, d_week_seq#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(119) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#112] -Condition : ((isnotnull(d_week_seq#112) AND (d_week_seq#112 = Subquery scalar-subquery#113, [id=#114])) AND isnotnull(d_date_sk#65)) +(127) CometFilter +Input [2]: [d_date_sk#63, d_week_seq#107] +Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#63)) -(120) CometProject -Input [2]: [d_date_sk#65, d_week_seq#112] -Arguments: [d_date_sk#65], [d_date_sk#65] +(128) CometProject +Input [2]: [d_date_sk#63, d_week_seq#107] +Arguments: [d_date_sk#63], [d_date_sk#63] -(121) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] +(129) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#63] -(122) BroadcastExchange -Input [1]: [d_date_sk#65] +(130) BroadcastExchange +Input [1]: [d_date_sk#63] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:12 Hosting operator id = 119 Hosting Expression = Subquery scalar-subquery#113, [id=#114] -* ColumnarToRow (126) -+- CometProject (125) - +- CometFilter (124) - +- CometScan parquet spark_catalog.default.date_dim (123) +Subquery:12 Hosting operator id = 127 Hosting Expression = Subquery scalar-subquery#108, [id=#109] +* ColumnarToRow (134) ++- CometProject (133) + +- CometFilter (132) + +- CometScan parquet spark_catalog.default.date_dim (131) (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] +Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(124) CometFilter -Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] -Condition : (((((isnotnull(d_year#116) AND isnotnull(d_moy#117)) AND isnotnull(d_dom#118)) AND (d_year#116 = 1999)) AND (d_moy#117 = 12)) AND (d_dom#118 = 11)) +(132) CometFilter +Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] +Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1999)) AND (d_moy#112 = 12)) AND (d_dom#113 = 11)) -(125) CometProject -Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] -Arguments: [d_week_seq#115], [d_week_seq#115] +(133) CometProject +Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] +Arguments: [d_week_seq#110], [d_week_seq#110] -(126) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#115] +(134) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#110] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 09d8d9dde3..13ac319fbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -1,202 +1,210 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (52) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #4 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [sales] + Subquery #4 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometHashAggregate [sum,count] + CometColumnarExchange #12 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + RowToColumnar + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - ColumnarToRow - InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #5 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #6 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 InputAdapter ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #3 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 WholeStageCodegen (51) - Filter [sales] - ReusedSubquery [average_sales] #4 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow + ColumnarToRow + InputAdapter + CometFilter [sales] + ReusedSubquery [average_sales] #4 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 + RowToColumnar + WholeStageCodegen (50) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + BroadcastExchange #15 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + Subquery #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + ReusedExchange [ss_item_sk] #3 InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] - Subquery #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #15 + ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt index 811a9eef51..135162ca9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt @@ -1,26 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.customer (4) - : +- BroadcastExchange (13) - : +- * ColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.customer_address (10) - +- ReusedExchange (16) +* ColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometHashAggregate (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (15) + : +- * BroadcastHashJoin Inner BuildRight (14) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.customer (4) + : +- BroadcastExchange (13) + : +- * ColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.customer_address (10) + +- ReusedExchange (16) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -94,7 +96,7 @@ Join condition: ((substr(ca_zip#9, 1, 5) IN (85669,86197,88274,83405,86475,85392 Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9] Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#8, ca_zip#9] -(16) ReusedExchange [Reuses operator id: 27] +(16) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#10] (17) BroadcastHashJoin [codegen id : 4] @@ -114,50 +116,54 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] Aggregate Attributes [1]: [sum#11] Results [2]: [ca_zip#9, sum#12] -(20) Exchange +(20) RowToColumnar Input [2]: [ca_zip#9, sum#12] -Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) HashAggregate [codegen id : 5] +(21) CometColumnarExchange +Input [2]: [ca_zip#9, sum#12] +Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) CometHashAggregate Input [2]: [ca_zip#9, sum#12] Keys [1]: [ca_zip#9] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#13] -Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#13,17,2) AS sum(cs_sales_price)#14] -(22) TakeOrderedAndProject -Input [2]: [ca_zip#9, sum(cs_sales_price)#14] -Arguments: 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#14] +(23) CometTakeOrderedAndProject +Input [2]: [ca_zip#9, sum(cs_sales_price)#13] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#9 ASC NULLS FIRST], output=[ca_zip#9,sum(cs_sales_price)#13]), 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#13] + +(24) ColumnarToRow [codegen id : 5] +Input [2]: [ca_zip#9, sum(cs_sales_price)#13] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#15, d_qoy#16] +Output [3]: [d_date_sk#10, d_year#14, d_qoy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] -Condition : ((((isnotnull(d_qoy#16) AND isnotnull(d_year#15)) AND (d_qoy#16 = 2)) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#10)) +(26) CometFilter +Input [3]: [d_date_sk#10, d_year#14, d_qoy#15] +Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#10)) -(25) CometProject -Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] +(27) CometProject +Input [3]: [d_date_sk#10, d_year#14, d_qoy#15] Arguments: [d_date_sk#10], [d_date_sk#10] -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index 5c750b2db8..58d3742879 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -1,41 +1,43 @@ -TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (5) - HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - InputAdapter - Exchange [ca_zip] #1 - WholeStageCodegen (4) - HashAggregate [ca_zip,cs_sales_price] [sum,sum] - Project [cs_sales_price,ca_zip] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] - Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] +WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] + CometHashAggregate [ca_zip,sum] + CometColumnarExchange [ca_zip] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ca_zip,cs_sales_price] [sum,sum] + Project [cs_sales_price,ca_zip] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sales_price,cs_sold_date_sk,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 1762c8c6c8..cb01af81df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -* HashAggregate (45) -+- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- * ColumnarToRow (11) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) - : : : +- * ColumnarToRow (18) - : : : +- CometSort (17) - : : : +- CometExchange (16) - : : : +- CometProject (15) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.call_center (34) +* ColumnarToRow (47) ++- CometHashAggregate (46) + +- CometColumnarExchange (45) + +- RowToColumnar (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- * SortMergeJoin LeftSemi (12) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- * ColumnarToRow (11) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) + : : : +- * ColumnarToRow (18) + : : : +- CometSort (17) + : : : +- CometColumnarExchange (16) + : : : +- CometProject (15) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometProject (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.date_dim (20) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.call_center (34) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -61,9 +63,9 @@ Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(4) CometExchange +(4) CometColumnarExchange Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] (5) CometSort Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -82,9 +84,9 @@ ReadSchema: struct Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] -(9) CometExchange +(9) CometColumnarExchange Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (10) CometSort Input [2]: [cs_warehouse_sk#9, cs_order_number#10] @@ -113,9 +115,9 @@ ReadSchema: struct Input [2]: [cr_order_number#12, cr_returned_date_sk#13] Arguments: [cr_order_number#12], [cr_order_number#12] -(16) CometExchange +(16) CometColumnarExchange Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (17) CometSort Input [1]: [cr_order_number#12] @@ -247,14 +249,18 @@ Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(Unscaled Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) Exchange +(44) RowToColumnar Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 9] +(45) CometColumnarExchange +Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(46) CometHashAggregate Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24] -Results [3]: [count(cs_order_number#5)#24 AS order count #26, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#20,17,2) AS total shipping cost #27, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#21,17,2) AS total net profit #28] + +(47) ColumnarToRow [codegen id : 9] +Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index 213726372e..eed6c6d22d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -1,68 +1,70 @@ WholeStageCodegen (9) - HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (8) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] - Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - SortMergeJoin [cs_order_number,cr_order_number] - InputAdapter - WholeStageCodegen (3) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - InputAdapter - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometSort [cs_order_number] - CometExchange [cs_order_number] #2 - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometSort [cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + CometHashAggregate [sum,sum,count] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] + Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + SortMergeJoin [cs_order_number,cr_order_number] + InputAdapter + WholeStageCodegen (3) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [cs_order_number] + CometColumnarExchange [cs_order_number] #2 + CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [cs_order_number] + CometColumnarExchange [cs_order_number] #3 + CometProject [cs_warehouse_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [cr_order_number] + CometColumnarExchange [cr_order_number] #4 + CometProject [cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] + CometProject [cc_call_center_sk] + CometFilter [cc_county,cc_call_center_sk] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index 5e9b6d1da7..759cb2384e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -1,44 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- * ColumnarToRow (40) + +- CometColumnarExchange (39) + +- RowToColumnar (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (18) + : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) + : : : : +- ReusedExchange (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.store (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.item (31) (unknown) Scan parquet spark_catalog.default.store_sales @@ -114,7 +116,7 @@ Join condition: None Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -(16) ReusedExchange [Reuses operator id: 45] +(16) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#19] (17) BroadcastHashJoin [codegen id : 8] @@ -127,7 +129,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -(19) ReusedExchange [Reuses operator id: 50] +(19) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#20] (20) BroadcastHashJoin [codegen id : 8] @@ -140,7 +142,7 @@ Join condition: None Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] -(22) ReusedExchange [Reuses operator id: 50] +(22) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#21] (23) BroadcastHashJoin [codegen id : 8] @@ -216,29 +218,35 @@ Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partia Aggregate Attributes [18]: [count#27, sum#28, count#29, n#30, avg#31, m2#32, count#33, sum#34, count#35, n#36, avg#37, m2#38, count#39, sum#40, count#41, n#42, avg#43, m2#44] Results [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] -(38) Exchange +(38) RowToColumnar Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] -Arguments: hashpartitioning(i_item_id#25, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(39) HashAggregate [codegen id : 9] +(39) CometColumnarExchange +Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] +Arguments: hashpartitioning(i_item_id#25, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(40) ColumnarToRow [codegen id : 9] +Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] + +(41) HashAggregate [codegen id : 9] Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] Keys [3]: [i_item_id#25, i_item_desc#26, s_state#23] Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] Aggregate Attributes [9]: [count(ss_quantity#5)#63, avg(ss_quantity#5)#64, stddev_samp(cast(ss_quantity#5 as double))#65, count(sr_return_quantity#11)#66, avg(sr_return_quantity#11)#67, stddev_samp(cast(sr_return_quantity#11 as double))#68, count(cs_quantity#16)#69, avg(cs_quantity#16)#70, stddev_samp(cast(cs_quantity#16 as double))#71] Results [15]: [i_item_id#25, i_item_desc#26, s_state#23, count(ss_quantity#5)#63 AS store_sales_quantitycount#72, avg(ss_quantity#5)#64 AS store_sales_quantityave#73, stddev_samp(cast(ss_quantity#5 as double))#65 AS store_sales_quantitystdev#74, (stddev_samp(cast(ss_quantity#5 as double))#65 / avg(ss_quantity#5)#64) AS store_sales_quantitycov#75, count(sr_return_quantity#11)#66 AS as_store_returns_quantitycount#76, avg(sr_return_quantity#11)#67 AS as_store_returns_quantityave#77, stddev_samp(cast(sr_return_quantity#11 as double))#68 AS as_store_returns_quantitystdev#78, (stddev_samp(cast(sr_return_quantity#11 as double))#68 / avg(sr_return_quantity#11)#67) AS store_returns_quantitycov#79, count(cs_quantity#16)#69 AS catalog_sales_quantitycount#80, avg(cs_quantity#16)#70 AS catalog_sales_quantityave#81, (stddev_samp(cast(cs_quantity#16 as double))#71 / avg(cs_quantity#16)#70) AS catalog_sales_quantitystdev#82, (stddev_samp(cast(cs_quantity#16 as double))#71 / avg(cs_quantity#16)#70) AS catalog_sales_quantitycov#83] -(40) TakeOrderedAndProject +(42) TakeOrderedAndProject Input [15]: [i_item_id#25, i_item_desc#26, s_state#23, store_sales_quantitycount#72, store_sales_quantityave#73, store_sales_quantitystdev#74, store_sales_quantitycov#75, as_store_returns_quantitycount#76, as_store_returns_quantityave#77, as_store_returns_quantitystdev#78, store_returns_quantitycov#79, catalog_sales_quantitycount#80, catalog_sales_quantityave#81, catalog_sales_quantitystdev#82, catalog_sales_quantitycov#83] Arguments: 100, [i_item_id#25 ASC NULLS FIRST, i_item_desc#26 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#25, i_item_desc#26, s_state#23, store_sales_quantitycount#72, store_sales_quantityave#73, store_sales_quantitystdev#74, store_sales_quantitycov#75, as_store_returns_quantitycount#76, as_store_returns_quantityave#77, as_store_returns_quantitystdev#78, store_returns_quantitycov#79, catalog_sales_quantitycount#80, catalog_sales_quantityave#81, catalog_sales_quantitystdev#82, catalog_sales_quantitycov#83] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (unknown) Scan parquet spark_catalog.default.date_dim @@ -248,27 +256,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter +(44) CometFilter Input [2]: [d_date_sk#19, d_quarter_name#84] Condition : ((isnotnull(d_quarter_name#84) AND (d_quarter_name#84 = 2001Q1)) AND isnotnull(d_date_sk#19)) -(43) CometProject +(45) CometProject Input [2]: [d_date_sk#19, d_quarter_name#84] Arguments: [d_date_sk#19], [d_date_sk#19] -(44) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(45) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim @@ -278,18 +286,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(49) CometFilter Input [2]: [d_date_sk#20, d_quarter_name#85] Condition : (d_quarter_name#85 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#20)) -(48) CometProject +(50) CometProject Input [2]: [d_date_sk#20, d_quarter_name#85] Arguments: [d_date_sk#20], [d_date_sk#20] -(49) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#20] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index 9f4d67decc..13d7ab8775 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -1,76 +1,78 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] WholeStageCodegen (9) HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - InputAdapter - Exchange [i_item_id,i_item_desc,s_state] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,s_state] #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] + Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #4 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #4 + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt index 613377a61a..159a82afde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt @@ -1,47 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Expand (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Project (17) - : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- CometProject (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : : +- BroadcastExchange (15) - : : : : +- * ColumnarToRow (14) - : : : : +- CometProject (13) - : : : : +- CometFilter (12) - : : : : +- CometScan parquet spark_catalog.default.customer (11) - : : : +- BroadcastExchange (21) - : : : +- * ColumnarToRow (20) - : : : +- CometFilter (19) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) - : : +- BroadcastExchange (27) - : : +- * ColumnarToRow (26) - : : +- CometFilter (25) - : : +- CometScan parquet spark_catalog.default.customer_address (24) - : +- ReusedExchange (30) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.item (33) +* ColumnarToRow (45) ++- CometTakeOrderedAndProject (44) + +- CometHashAggregate (43) + +- CometColumnarExchange (42) + +- RowToColumnar (41) + +- * HashAggregate (40) + +- * Expand (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * Project (17) + : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : :- * Project (10) + : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : +- BroadcastExchange (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- CometProject (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : : : +- BroadcastExchange (15) + : : : : +- * ColumnarToRow (14) + : : : : +- CometProject (13) + : : : : +- CometFilter (12) + : : : : +- CometScan parquet spark_catalog.default.customer (11) + : : : +- BroadcastExchange (21) + : : : +- * ColumnarToRow (20) + : : : +- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) + : : +- BroadcastExchange (27) + : : +- * ColumnarToRow (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.customer_address (24) + : +- ReusedExchange (30) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.item (33) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -179,7 +181,7 @@ Join condition: None Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -(30) ReusedExchange [Reuses operator id: 48] +(30) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 7] @@ -231,50 +233,54 @@ Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(c Aggregate Attributes [14]: [sum#33, count#34, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46] Results [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] -(41) Exchange +(41) RowToColumnar Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] -Arguments: hashpartitioning(i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(42) HashAggregate [codegen id : 8] +(42) CometColumnarExchange +Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] +Arguments: hashpartitioning(i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(43) CometHashAggregate Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] Keys [5]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] -Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#61, avg(cast(cs_list_price#5 as decimal(12,2)))#62, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63, avg(cast(cs_sales_price#6 as decimal(12,2)))#64, avg(cast(cs_net_profit#8 as decimal(12,2)))#65, avg(cast(c_birth_year#19 as decimal(12,2)))#66, avg(cast(cd_dep_count#14 as decimal(12,2)))#67] -Results [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, avg(cast(cs_quantity#4 as decimal(12,2)))#61 AS agg1#68, avg(cast(cs_list_price#5 as decimal(12,2)))#62 AS agg2#69, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63 AS agg3#70, avg(cast(cs_sales_price#6 as decimal(12,2)))#64 AS agg4#71, avg(cast(cs_net_profit#8 as decimal(12,2)))#65 AS agg5#72, avg(cast(c_birth_year#19 as decimal(12,2)))#66 AS agg6#73, avg(cast(cd_dep_count#14 as decimal(12,2)))#67 AS agg7#74] -(43) TakeOrderedAndProject -Input [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] -Arguments: 100, [ca_country#29 ASC NULLS FIRST, ca_state#30 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] +(44) CometTakeOrderedAndProject +Input [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#61, agg2#62, agg3#63, agg4#64, agg5#65, agg6#66, agg7#67] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#29 ASC NULLS FIRST,ca_state#30 ASC NULLS FIRST,ca_county#31 ASC NULLS FIRST,i_item_id#28 ASC NULLS FIRST], output=[i_item_id#28,ca_country#29,ca_state#30,ca_county#31,agg1#61,agg2#62,agg3#63,agg4#64,agg5#65,agg6#66,agg7#67]), 100, [ca_country#29 ASC NULLS FIRST, ca_state#30 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#61, agg2#62, agg3#63, agg4#64, agg5#65, agg6#66, agg7#67] + +(45) ColumnarToRow [codegen id : 8] +Input [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#61, agg2#62, agg3#63, agg4#64, agg5#65, agg6#66, agg7#67] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#75] +Output [2]: [d_date_sk#25, d_year#68] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [2]: [d_date_sk#25, d_year#75] -Condition : ((isnotnull(d_year#75) AND (d_year#75 = 1998)) AND isnotnull(d_date_sk#25)) +(47) CometFilter +Input [2]: [d_date_sk#25, d_year#68] +Condition : ((isnotnull(d_year#68) AND (d_year#68 = 1998)) AND isnotnull(d_date_sk#25)) -(46) CometProject -Input [2]: [d_date_sk#25, d_year#75] +(48) CometProject +Input [2]: [d_date_sk#25, d_year#68] Arguments: [d_date_sk#25], [d_date_sk#25] -(47) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(48) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt index 47911b9ba3..0f94dd57c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt @@ -1,71 +1,73 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - WholeStageCodegen (8) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Expand [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #6 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt index 1150b3d665..5f4b64a18d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt @@ -1,43 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (17) - : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.date_dim (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan parquet spark_catalog.default.store_sales (5) - : : : +- BroadcastExchange (15) - : : : +- * ColumnarToRow (14) - : : : +- CometProject (13) - : : : +- CometFilter (12) - : : : +- CometScan parquet spark_catalog.default.item (11) - : : +- BroadcastExchange (21) - : : +- * ColumnarToRow (20) - : : +- CometFilter (19) - : : +- CometScan parquet spark_catalog.default.customer (18) - : +- BroadcastExchange (27) - : +- * ColumnarToRow (26) - : +- CometFilter (25) - : +- CometScan parquet spark_catalog.default.customer_address (24) - +- BroadcastExchange (33) - +- * ColumnarToRow (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.store (30) +* ColumnarToRow (41) ++- CometTakeOrderedAndProject (40) + +- CometHashAggregate (39) + +- CometColumnarExchange (38) + +- RowToColumnar (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) + : :- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Project (17) + : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometFilter (6) + : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : +- BroadcastExchange (15) + : : : +- * ColumnarToRow (14) + : : : +- CometProject (13) + : : : +- CometFilter (12) + : : : +- CometScan parquet spark_catalog.default.item (11) + : : +- BroadcastExchange (21) + : : +- * ColumnarToRow (20) + : : +- CometFilter (19) + : : +- CometScan parquet spark_catalog.default.customer (18) + : +- BroadcastExchange (27) + : +- * ColumnarToRow (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer_address (24) + +- BroadcastExchange (33) + +- * ColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.store (30) (unknown) Scan parquet spark_catalog.default.date_dim @@ -210,18 +212,22 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] Aggregate Attributes [1]: [sum#21] Results [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] -(37) Exchange +(37) RowToColumnar Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] -Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(38) HashAggregate [codegen id : 7] +(38) CometColumnarExchange +Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] +Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(39) CometHashAggregate Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] Keys [4]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#23] -Results [5]: [i_brand_id#10 AS brand_id#24, i_brand#11 AS brand#25, i_manufact_id#12, i_manufact#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#23,17,2) AS ext_price#26] -(39) TakeOrderedAndProject -Input [5]: [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] -Arguments: 100, [ext_price#26 DESC NULLS LAST, brand#25 ASC NULLS FIRST, brand_id#24 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] +(40) CometTakeOrderedAndProject +Input [5]: [brand_id#23, brand#24, i_manufact_id#12, i_manufact#13, ext_price#25] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#25 DESC NULLS LAST,brand#24 ASC NULLS FIRST,brand_id#23 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#13 ASC NULLS FIRST], output=[brand_id#23,brand#24,i_manufact_id#12,i_manufact#13,ext_price#25]), 100, [ext_price#25 DESC NULLS LAST, brand#24 ASC NULLS FIRST, brand_id#23 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#23, brand#24, i_manufact_id#12, i_manufact#13, ext_price#25] + +(41) ColumnarToRow [codegen id : 7] +Input [5]: [brand_id#23, brand#24, i_manufact_id#12, i_manufact#13, ext_price#25] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt index c2f5d1a876..5b9f4956e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt @@ -1,58 +1,60 @@ -TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] - WholeStageCodegen (7) - HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - InputAdapter - Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - WholeStageCodegen (6) - HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] - Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] + CometColumnarExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #6 + WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [s_zip,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] + CometFilter [s_zip,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt index 2fea53fa79..7974fb4eeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt @@ -1,40 +1,45 @@ == Physical Plan == -* Sort (36) -+- Exchange (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * ColumnarToRow (6) - : : : +- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.date_dim (7) - : +- BroadcastExchange (20) - : +- * ColumnarToRow (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.date_dim (16) - +- BroadcastExchange (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * HashAggregate (24) - : +- ReusedExchange (23) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +* ColumnarToRow (41) ++- CometSort (40) + +- CometColumnarExchange (39) + +- RowToColumnar (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * ColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometColumnarExchange (15) + : : +- RowToColumnar (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * ColumnarToRow (6) + : : : +- CometUnion (5) + : : : :- CometProject (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- CometProject (4) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.date_dim (7) + : +- BroadcastExchange (22) + : +- * ColumnarToRow (21) + : +- CometProject (20) + : +- CometFilter (19) + : +- CometScan parquet spark_catalog.default.date_dim (18) + +- BroadcastExchange (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * ColumnarToRow (27) + : +- CometHashAggregate (26) + : +- ReusedExchange (25) + +- BroadcastExchange (32) + +- * ColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan parquet spark_catalog.default.date_dim (28) (unknown) Scan parquet spark_catalog.default.web_sales @@ -101,110 +106,121 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) Aggregate Attributes [7]: [sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18] Results [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] -(14) Exchange +(14) RowToColumnar Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 8] +(15) CometColumnarExchange +Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] +Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26,17,2) AS sun_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27,17,2) AS mon_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28,17,2) AS tue_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29,17,2) AS wed_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30,17,2) AS thu_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31,17,2) AS fri_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32,17,2) AS sat_sales#39] + +(17) ColumnarToRow [codegen id : 8] +Input [8]: [d_week_seq#10, sun_sales#26, mon_sales#27, tue_sales#28, wed_sales#29, thu_sales#30, fri_sales#31, sat_sales#32] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#40, d_year#41] +Output [2]: [d_week_seq#33, d_year#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] ReadSchema: struct -(17) CometFilter -Input [2]: [d_week_seq#40, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2001)) AND isnotnull(d_week_seq#40)) +(19) CometFilter +Input [2]: [d_week_seq#33, d_year#34] +Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2001)) AND isnotnull(d_week_seq#33)) -(18) CometProject -Input [2]: [d_week_seq#40, d_year#41] -Arguments: [d_week_seq#40], [d_week_seq#40] +(20) CometProject +Input [2]: [d_week_seq#33, d_year#34] +Arguments: [d_week_seq#33], [d_week_seq#33] -(19) ColumnarToRow [codegen id : 3] -Input [1]: [d_week_seq#40] +(21) ColumnarToRow [codegen id : 3] +Input [1]: [d_week_seq#33] -(20) BroadcastExchange -Input [1]: [d_week_seq#40] +(22) BroadcastExchange +Input [1]: [d_week_seq#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(21) BroadcastHashJoin [codegen id : 8] +(23) BroadcastHashJoin [codegen id : 8] Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#40] +Right keys [1]: [d_week_seq#33] Join type: Inner Join condition: None -(22) Project [codegen id : 8] -Output [8]: [d_week_seq#10 AS d_week_seq1#42, sun_sales#33 AS sun_sales1#43, mon_sales#34 AS mon_sales1#44, tue_sales#35 AS tue_sales1#45, wed_sales#36 AS wed_sales1#46, thu_sales#37 AS thu_sales1#47, fri_sales#38 AS fri_sales1#48, sat_sales#39 AS sat_sales1#49] -Input [9]: [d_week_seq#10, sun_sales#33, mon_sales#34, tue_sales#35, wed_sales#36, thu_sales#37, fri_sales#38, sat_sales#39, d_week_seq#40] +(24) Project [codegen id : 8] +Output [8]: [d_week_seq#10 AS d_week_seq1#35, sun_sales#26 AS sun_sales1#36, mon_sales#27 AS mon_sales1#37, tue_sales#28 AS tue_sales1#38, wed_sales#29 AS wed_sales1#39, thu_sales#30 AS thu_sales1#40, fri_sales#31 AS fri_sales1#41, sat_sales#32 AS sat_sales1#42] +Input [9]: [d_week_seq#10, sun_sales#26, mon_sales#27, tue_sales#28, wed_sales#29, thu_sales#30, fri_sales#31, sat_sales#32, d_week_seq#33] -(23) ReusedExchange [Reuses operator id: 14] -Output [8]: [d_week_seq#10, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] +(25) ReusedExchange [Reuses operator id: 15] +Output [8]: [d_week_seq#10, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] -(24) HashAggregate [codegen id : 7] -Input [8]: [d_week_seq#10, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] +(26) CometHashAggregate +Input [8]: [d_week_seq#10, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26,17,2) AS sun_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27,17,2) AS mon_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28,17,2) AS tue_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29,17,2) AS wed_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30,17,2) AS thu_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31,17,2) AS fri_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32,17,2) AS sat_sales#39] + +(27) ColumnarToRow [codegen id : 7] +Input [8]: [d_week_seq#10, sun_sales#26, mon_sales#27, tue_sales#28, wed_sales#29, thu_sales#30, fri_sales#31, sat_sales#32] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#57, d_year#58] +Output [2]: [d_week_seq#50, d_year#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct -(26) CometFilter -Input [2]: [d_week_seq#57, d_year#58] -Condition : ((isnotnull(d_year#58) AND (d_year#58 = 2002)) AND isnotnull(d_week_seq#57)) +(29) CometFilter +Input [2]: [d_week_seq#50, d_year#51] +Condition : ((isnotnull(d_year#51) AND (d_year#51 = 2002)) AND isnotnull(d_week_seq#50)) -(27) CometProject -Input [2]: [d_week_seq#57, d_year#58] -Arguments: [d_week_seq#57], [d_week_seq#57] +(30) CometProject +Input [2]: [d_week_seq#50, d_year#51] +Arguments: [d_week_seq#50], [d_week_seq#50] -(28) ColumnarToRow [codegen id : 6] -Input [1]: [d_week_seq#57] +(31) ColumnarToRow [codegen id : 6] +Input [1]: [d_week_seq#50] -(29) BroadcastExchange -Input [1]: [d_week_seq#57] +(32) BroadcastExchange +Input [1]: [d_week_seq#50] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(30) BroadcastHashJoin [codegen id : 7] +(33) BroadcastHashJoin [codegen id : 7] Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#57] +Right keys [1]: [d_week_seq#50] Join type: Inner Join condition: None -(31) Project [codegen id : 7] -Output [8]: [d_week_seq#10 AS d_week_seq2#59, sun_sales#33 AS sun_sales2#60, mon_sales#34 AS mon_sales2#61, tue_sales#35 AS tue_sales2#62, wed_sales#36 AS wed_sales2#63, thu_sales#37 AS thu_sales2#64, fri_sales#38 AS fri_sales2#65, sat_sales#39 AS sat_sales2#66] -Input [9]: [d_week_seq#10, sun_sales#33, mon_sales#34, tue_sales#35, wed_sales#36, thu_sales#37, fri_sales#38, sat_sales#39, d_week_seq#57] +(34) Project [codegen id : 7] +Output [8]: [d_week_seq#10 AS d_week_seq2#52, sun_sales#26 AS sun_sales2#53, mon_sales#27 AS mon_sales2#54, tue_sales#28 AS tue_sales2#55, wed_sales#29 AS wed_sales2#56, thu_sales#30 AS thu_sales2#57, fri_sales#31 AS fri_sales2#58, sat_sales#32 AS sat_sales2#59] +Input [9]: [d_week_seq#10, sun_sales#26, mon_sales#27, tue_sales#28, wed_sales#29, thu_sales#30, fri_sales#31, sat_sales#32, d_week_seq#50] -(32) BroadcastExchange -Input [8]: [d_week_seq2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] +(35) BroadcastExchange +Input [8]: [d_week_seq2#52, sun_sales2#53, mon_sales2#54, tue_sales2#55, wed_sales2#56, thu_sales2#57, fri_sales2#58, sat_sales2#59] Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [d_week_seq1#42] -Right keys [1]: [(d_week_seq2#59 - 53)] +(36) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [d_week_seq1#35] +Right keys [1]: [(d_week_seq2#52 - 53)] Join type: Inner Join condition: None -(34) Project [codegen id : 8] -Output [8]: [d_week_seq1#42, round((sun_sales1#43 / sun_sales2#60), 2) AS round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1#44 / mon_sales2#61), 2) AS round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1#45 / tue_sales2#62), 2) AS round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1#46 / wed_sales2#63), 2) AS round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1#47 / thu_sales2#64), 2) AS round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1#48 / fri_sales2#65), 2) AS round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1#49 / sat_sales2#66), 2) AS round((sat_sales1 / sat_sales2), 2)#73] -Input [16]: [d_week_seq1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] +(37) Project [codegen id : 8] +Output [8]: [d_week_seq1#35, round((sun_sales1#36 / sun_sales2#53), 2) AS round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1#37 / mon_sales2#54), 2) AS round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1#38 / tue_sales2#55), 2) AS round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1#39 / wed_sales2#56), 2) AS round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1#40 / thu_sales2#57), 2) AS round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1#41 / fri_sales2#58), 2) AS round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1#42 / sat_sales2#59), 2) AS round((sat_sales1 / sat_sales2), 2)#66] +Input [16]: [d_week_seq1#35, sun_sales1#36, mon_sales1#37, tue_sales1#38, wed_sales1#39, thu_sales1#40, fri_sales1#41, sat_sales1#42, d_week_seq2#52, sun_sales2#53, mon_sales2#54, tue_sales2#55, wed_sales2#56, thu_sales2#57, fri_sales2#58, sat_sales2#59] + +(38) RowToColumnar +Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66] + +(39) CometColumnarExchange +Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66] +Arguments: rangepartitioning(d_week_seq1#35 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(35) Exchange -Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] -Arguments: rangepartitioning(d_week_seq1#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(40) CometSort +Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66] +Arguments: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66], [d_week_seq1#35 ASC NULLS FIRST] -(36) Sort [codegen id : 9] -Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] -Arguments: [d_week_seq1#42 ASC NULLS FIRST], true, 0 +(41) ColumnarToRow [codegen id : 9] +Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index 8856ce80d2..f4b68cfee2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -1,54 +1,59 @@ WholeStageCodegen (9) - Sort [d_week_seq1] + ColumnarToRow InputAdapter - Exchange [d_week_seq1] #1 - WholeStageCodegen (8) - Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [d_week_seq1,d_week_seq2] - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - InputAdapter - Exchange [d_week_seq] #2 - WholeStageCodegen (2) - HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] - CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] - CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] - CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow + CometSort [d_week_seq1] + CometColumnarExchange [d_week_seq1] #1 + RowToColumnar + WholeStageCodegen (8) + Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] + BroadcastHashJoin [d_week_seq1,d_week_seq2] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + BroadcastHashJoin [d_week_seq,d_week_seq] + ColumnarToRow + InputAdapter + CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] + CometColumnarExchange [d_week_seq] #2 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] + CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] + CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] + CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - InputAdapter - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + BroadcastExchange #5 + WholeStageCodegen (7) + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + BroadcastHashJoin [d_week_seq,d_week_seq] + ColumnarToRow + InputAdapter + CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] + ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index eedf666dd1..5d279243c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -64,7 +66,7 @@ Join condition: None Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] +(10) ReusedExchange [Reuses operator id: 27] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -84,66 +86,70 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(14) RowToColumnar Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(15) CometColumnarExchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] -Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] -(16) Exchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(17) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(18) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(17) Sort [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [i_class#9 ASC NULLS FIRST], false, 0 +(19) ColumnarToRow [codegen id : 4] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] -(18) Window -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(20) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(19) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] -Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] +(21) Project [codegen id : 5] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] -(20) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +(22) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +Output [2]: [d_date_sk#11, d_date#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#18] +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#18] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index 52c42bdf2b..202a09b6d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -1,16 +1,16 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (5) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - Sort [i_class] + WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometSort [i_class] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index b5625d8e03..5c69eb848b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -1,28 +1,30 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Filter (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.warehouse (4) - : +- BroadcastExchange (14) - : +- * ColumnarToRow (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.item (10) - +- ReusedExchange (17) +* ColumnarToRow (26) ++- CometTakeOrderedAndProject (25) + +- CometFilter (24) + +- CometHashAggregate (23) + +- CometColumnarExchange (22) + +- RowToColumnar (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.warehouse (4) + : +- BroadcastExchange (14) + : +- * ColumnarToRow (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.item (10) + +- ReusedExchange (17) (unknown) Scan parquet spark_catalog.default.inventory @@ -100,7 +102,7 @@ Join condition: None Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9] Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#9] -(17) ReusedExchange [Reuses operator id: 28] +(17) ReusedExchange [Reuses operator id: 30] Output [2]: [d_date_sk#11, d_date#12] (18) BroadcastHashJoin [codegen id : 4] @@ -120,32 +122,36 @@ Functions [2]: [partial_sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity Aggregate Attributes [2]: [sum#13, sum#14] Results [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] -(21) Exchange +(21) RowToColumnar Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) HashAggregate [codegen id : 5] +(22) CometColumnarExchange +Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] +Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(23) CometHashAggregate Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] Keys [2]: [w_warehouse_name#7, i_item_id#9] Functions [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] -Aggregate Attributes [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18] -Results [4]: [w_warehouse_name#7, i_item_id#9, sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17 AS inv_before#19, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_after#20] -(23) Filter [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] -Condition : (CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667) END AND CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5) END) +(24) CometFilter +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18] +Condition : (CASE WHEN (inv_before#17 > 0) THEN ((cast(inv_after#18 as double) / cast(inv_before#17 as double)) >= 0.666667) END AND CASE WHEN (inv_before#17 > 0) THEN ((cast(inv_after#18 as double) / cast(inv_before#17 as double)) <= 1.5) END) + +(25) CometTakeOrderedAndProject +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#9 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#9,inv_before#17,inv_after#18]), 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18] -(24) TakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] -Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] +(26) ColumnarToRow [codegen id : 5] +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * ColumnarToRow (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) (unknown) Scan parquet spark_catalog.default.date_dim @@ -155,14 +161,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] ReadSchema: struct -(26) CometFilter +(28) CometFilter Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-10)) AND (d_date#12 <= 2000-04-10)) AND isnotnull(d_date_sk#11)) -(27) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_date#12] -(28) BroadcastExchange +(30) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index e20755e12f..3556650ca1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -1,42 +1,44 @@ -TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - WholeStageCodegen (5) - Filter [inv_before,inv_after] - HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] - InputAdapter - Exchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_warehouse_sk,inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] + CometFilter [inv_before,inv_after] + CometHashAggregate [w_warehouse_name,i_item_id,sum,sum] + CometColumnarExchange [w_warehouse_name,i_item_id] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] + Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_warehouse_sk,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometProject [i_item_sk,i_item_id] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #2 + ReusedExchange [d_date_sk,d_date] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt index 7dfa2dc06a..13689dc21e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt @@ -1,27 +1,29 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.warehouse (13) +* ColumnarToRow (25) ++- CometTakeOrderedAndProject (24) + +- CometHashAggregate (23) + +- CometColumnarExchange (22) + +- RowToColumnar (21) + +- * HashAggregate (20) + +- * Expand (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + +- BroadcastExchange (16) + +- * ColumnarToRow (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.warehouse (13) (unknown) Scan parquet spark_catalog.default.inventory @@ -39,7 +41,7 @@ Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (3) ColumnarToRow [codegen id : 4] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(4) ReusedExchange [Reuses operator id: 28] +(4) ReusedExchange [Reuses operator id: 30] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 4] @@ -119,50 +121,54 @@ Functions [1]: [partial_avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [sum#18, count#19] Results [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] -(21) Exchange +(21) RowToColumnar Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] -Arguments: hashpartitioning(i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) HashAggregate [codegen id : 5] +(22) CometColumnarExchange +Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] +Arguments: hashpartitioning(i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(23) CometHashAggregate Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] Keys [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#22] -Results [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, avg(inv_quantity_on_hand#3)#22 AS qoh#23] -(23) TakeOrderedAndProject -Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] -Arguments: 100, [qoh#23 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] +(24) CometTakeOrderedAndProject +Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#22] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#22 ASC NULLS FIRST,i_product_name#13 ASC NULLS FIRST,i_brand#14 ASC NULLS FIRST,i_class#15 ASC NULLS FIRST,i_category#16 ASC NULLS FIRST], output=[i_product_name#13,i_brand#14,i_class#15,i_category#16,qoh#22]), 100, [qoh#22 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#22] + +(25) ColumnarToRow [codegen id : 5] +Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * ColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.date_dim (24) +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#24] +Output [2]: [d_date_sk#6, d_month_seq#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#24] -Condition : (((isnotnull(d_month_seq#24) AND (d_month_seq#24 >= 1200)) AND (d_month_seq#24 <= 1211)) AND isnotnull(d_date_sk#6)) +(27) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#23] +Condition : (((isnotnull(d_month_seq#23) AND (d_month_seq#23 >= 1200)) AND (d_month_seq#23 <= 1211)) AND isnotnull(d_date_sk#6)) -(26) CometProject -Input [2]: [d_date_sk#6, d_month_seq#24] +(28) CometProject +Input [2]: [d_date_sk#6, d_month_seq#23] Arguments: [d_date_sk#6], [d_date_sk#6] -(27) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(28) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt index 92714bb02d..b0c2728e3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt @@ -1,42 +1,44 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - InputAdapter - Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index 5ec1794c1b..973532dc28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -1,70 +1,79 @@ == Physical Plan == -* HashAggregate (66) -+- Exchange (65) - +- * HashAggregate (64) - +- Union (63) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * SortMergeJoin LeftSemi (41) - : : :- * Sort (24) - : : : +- Exchange (23) - : : : +- * Project (22) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (21) - : : : :- * ColumnarToRow (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (20) - : : : +- * Project (19) - : : : +- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- Exchange (16) - : : : +- * HashAggregate (15) - : : : +- * Project (14) - : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : :- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (12) - : : : +- * ColumnarToRow (11) - : : : +- CometFilter (10) - : : : +- CometScan parquet spark_catalog.default.item (9) - : : +- * Sort (40) - : : +- * Project (39) - : : +- * Filter (38) - : : +- * HashAggregate (37) - : : +- Exchange (36) - : : +- * HashAggregate (35) - : : +- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * ColumnarToRow (28) - : : : +- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometScan parquet spark_catalog.default.store_sales (25) - : : +- BroadcastExchange (32) - : : +- * ColumnarToRow (31) - : : +- CometFilter (30) - : : +- CometScan parquet spark_catalog.default.customer (29) - : +- ReusedExchange (43) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * SortMergeJoin LeftSemi (58) - : :- * Sort (52) - : : +- Exchange (51) - : : +- * Project (50) - : : +- * BroadcastHashJoin LeftSemi BuildRight (49) - : : :- * ColumnarToRow (47) - : : : +- CometScan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (48) - : +- * Sort (57) - : +- * Project (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- ReusedExchange (53) - +- ReusedExchange (60) +* ColumnarToRow (75) ++- CometHashAggregate (74) + +- CometColumnarExchange (73) + +- RowToColumnar (72) + +- * HashAggregate (71) + +- Union (70) + :- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * ColumnarToRow (47) + : : +- CometProject (46) + : : +- CometSortMergeJoin (45) + : : :- CometSort (27) + : : : +- CometColumnarExchange (26) + : : : +- RowToColumnar (25) + : : : +- * Project (24) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : : :- * ColumnarToRow (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (22) + : : : +- * ColumnarToRow (21) + : : : +- CometProject (20) + : : : +- CometFilter (19) + : : : +- CometHashAggregate (18) + : : : +- CometColumnarExchange (17) + : : : +- RowToColumnar (16) + : : : +- * HashAggregate (15) + : : : +- * Project (14) + : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : :- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (12) + : : : +- * ColumnarToRow (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.item (9) + : : +- CometSort (44) + : : +- CometProject (43) + : : +- CometFilter (42) + : : +- CometHashAggregate (41) + : : +- CometColumnarExchange (40) + : : +- RowToColumnar (39) + : : +- * HashAggregate (38) + : : +- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * ColumnarToRow (31) + : : : +- CometProject (30) + : : : +- CometFilter (29) + : : : +- CometScan parquet spark_catalog.default.store_sales (28) + : : +- BroadcastExchange (35) + : : +- * ColumnarToRow (34) + : : +- CometFilter (33) + : : +- CometScan parquet spark_catalog.default.customer (32) + : +- ReusedExchange (48) + +- * Project (69) + +- * BroadcastHashJoin Inner BuildRight (68) + :- * ColumnarToRow (66) + : +- CometProject (65) + : +- CometSortMergeJoin (64) + : :- CometSort (58) + : : +- CometColumnarExchange (57) + : : +- RowToColumnar (56) + : : +- * Project (55) + : : +- * BroadcastHashJoin LeftSemi BuildRight (54) + : : :- * ColumnarToRow (52) + : : : +- CometScan parquet spark_catalog.default.web_sales (51) + : : +- ReusedExchange (53) + : +- CometSort (63) + : +- CometProject (62) + : +- CometFilter (61) + : +- CometHashAggregate (60) + : +- ReusedExchange (59) + +- ReusedExchange (67) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -92,7 +101,7 @@ Condition : isnotnull(ss_item_sk#7) (5) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(6) ReusedExchange [Reuses operator id: 76] +(6) ReusedExchange [Reuses operator id: 85] Output [2]: [d_date_sk#10, d_date#11] (7) BroadcastHashJoin [codegen id : 3] @@ -140,431 +149,450 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(16) Exchange +(16) RowToColumnar Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) HashAggregate [codegen id : 4] +(17) CometColumnarExchange +Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] +Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(18) CometHashAggregate Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(18) Filter [codegen id : 4] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) +(19) CometFilter +Input [2]: [item_sk#17, cnt#18] +Condition : (cnt#18 > 4) -(19) Project [codegen id : 4] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] +(20) CometProject +Input [2]: [item_sk#17, cnt#18] +Arguments: [item_sk#17], [item_sk#17] -(20) BroadcastExchange -Input [1]: [item_sk#18] +(21) ColumnarToRow [codegen id : 4] +Input [1]: [item_sk#17] + +(22) BroadcastExchange +Input [1]: [item_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(21) BroadcastHashJoin [codegen id : 5] +(23) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] +Right keys [1]: [item_sk#17] Join type: LeftSemi Join condition: None -(22) Project [codegen id : 5] +(24) Project [codegen id : 5] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(23) Exchange +(25) RowToColumnar +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + +(26) CometColumnarExchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) Sort [codegen id : 6] +(27) CometSort Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#20) +(29) CometFilter +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#19) -(27) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] +(30) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -(28) ColumnarToRow [codegen id : 8] -Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] +(31) ColumnarToRow [codegen id : 7] +Input [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] (unknown) Scan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] +Output [1]: [c_customer_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(30) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) +(33) CometFilter +Input [1]: [c_customer_sk#23] +Condition : isnotnull(c_customer_sk#23) -(31) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#24] +(34) ColumnarToRow [codegen id : 6] +Input [1]: [c_customer_sk#23] -(32) BroadcastExchange -Input [1]: [c_customer_sk#24] +(35) BroadcastExchange +Input [1]: [c_customer_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#20] -Right keys [1]: [c_customer_sk#24] +(36) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ss_customer_sk#19] +Right keys [1]: [c_customer_sk#23] Join type: Inner Join condition: None -(34) Project [codegen id : 8] -Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] - -(35) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Keys [1]: [c_customer_sk#24] -Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [2]: [sum#25, isEmpty#26] -Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] - -(36) Exchange -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(37) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] - -(38) Filter [codegen id : 9] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) - -(39) Project [codegen id : 9] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] - -(40) Sort [codegen id : 9] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(41) SortMergeJoin [codegen id : 11] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#24] -Join type: LeftSemi -Join condition: None +(37) Project [codegen id : 7] +Output [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] + +(38) HashAggregate [codegen id : 7] +Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Keys [1]: [c_customer_sk#23] +Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] +Aggregate Attributes [2]: [sum#24, isEmpty#25] +Results [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(39) RowToColumnar +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(40) CometColumnarExchange +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(41) CometHashAggregate +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(42) CometFilter +Input [2]: [c_customer_sk#23, ssales#28] +Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#29, [id=#30]))) + +(43) CometProject +Input [2]: [c_customer_sk#23, ssales#28] +Arguments: [c_customer_sk#23], [c_customer_sk#23] -(42) Project [codegen id : 11] -Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +(44) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(45) CometSortMergeJoin +Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [c_customer_sk#23] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi + +(46) CometProject Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(43) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#33] +(47) ColumnarToRow [codegen id : 9] +Input [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(44) BroadcastHashJoin [codegen id : 11] +(48) ReusedExchange [Reuses operator id: 80] +Output [1]: [d_date_sk#31] + +(49) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#33] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(45) Project [codegen id : 11] -Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#34] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#33] +(50) Project [codegen id : 9] +Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#31] (unknown) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#39), dynamicpruningexpression(ws_sold_date_sk#39 IN dynamicpruning#40)] +PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] -Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +(52) ColumnarToRow [codegen id : 14] +Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(48) ReusedExchange [Reuses operator id: 20] -Output [1]: [item_sk#18] +(53) ReusedExchange [Reuses operator id: 22] +Output [1]: [item_sk#17] -(49) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [item_sk#18] +(54) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [item_sk#17] Join type: LeftSemi Join condition: None -(50) Project [codegen id : 16] -Output [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +(55) Project [codegen id : 14] +Output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(51) Exchange -Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -Arguments: hashpartitioning(ws_bill_customer_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(56) RowToColumnar +Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(52) Sort [codegen id : 17] -Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -Arguments: [ws_bill_customer_sk#36 ASC NULLS FIRST], false, 0 +(57) CometColumnarExchange +Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) ReusedExchange [Reuses operator id: 36] -Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(58) CometSort +Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34 ASC NULLS FIRST] -(54) HashAggregate [codegen id : 20] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] +(59) ReusedExchange [Reuses operator id: 40] +Output [3]: [c_customer_sk#23, sum#26, isEmpty#27] -(55) Filter [codegen id : 20] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) +(60) CometHashAggregate +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] -(56) Project [codegen id : 20] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] +(61) CometFilter +Input [2]: [c_customer_sk#23, ssales#28] +Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30]))) -(57) Sort [codegen id : 20] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 +(62) CometProject +Input [2]: [c_customer_sk#23, ssales#28] +Arguments: [c_customer_sk#23], [c_customer_sk#23] -(58) SortMergeJoin [codegen id : 22] -Left keys [1]: [ws_bill_customer_sk#36] -Right keys [1]: [c_customer_sk#24] -Join type: LeftSemi -Join condition: None +(63) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(64) CometSortMergeJoin +Left output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Right output [1]: [c_customer_sk#23] +Arguments: [ws_bill_customer_sk#34], [c_customer_sk#23], LeftSemi -(59) Project [codegen id : 22] -Output [3]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +(65) CometProject +Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Arguments: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(60) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#41] +(66) ColumnarToRow [codegen id : 18] +Input [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(61) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#39] -Right keys [1]: [d_date_sk#41] +(67) ReusedExchange [Reuses operator id: 80] +Output [1]: [d_date_sk#39] + +(68) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_sold_date_sk#37] +Right keys [1]: [d_date_sk#39] Join type: Inner Join condition: None -(62) Project [codegen id : 22] -Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#42] -Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#41] +(69) Project [codegen id : 18] +Output [1]: [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#40] +Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#39] -(63) Union +(70) Union -(64) HashAggregate [codegen id : 23] -Input [1]: [sales#34] +(71) HashAggregate [codegen id : 19] +Input [1]: [sales#32] Keys: [] -Functions [1]: [partial_sum(sales#34)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [2]: [sum#45, isEmpty#46] +Functions [1]: [partial_sum(sales#32)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [2]: [sum#43, isEmpty#44] + +(72) RowToColumnar +Input [2]: [sum#43, isEmpty#44] -(65) Exchange -Input [2]: [sum#45, isEmpty#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] +(73) CometColumnarExchange +Input [2]: [sum#43, isEmpty#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(66) HashAggregate [codegen id : 24] -Input [2]: [sum#45, isEmpty#46] +(74) CometHashAggregate +Input [2]: [sum#43, isEmpty#44] Keys: [] -Functions [1]: [sum(sales#34)] -Aggregate Attributes [1]: [sum(sales#34)#47] -Results [1]: [sum(sales#34)#47 AS sum(sales)#48] +Functions [1]: [sum(sales#32)] + +(75) ColumnarToRow [codegen id : 20] +Input [1]: [sum(sales)#45] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (71) -+- * ColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan parquet spark_catalog.default.date_dim (67) +BroadcastExchange (80) ++- * ColumnarToRow (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#33, d_year#49, d_moy#50] +Output [3]: [d_date_sk#31, d_year#46, d_moy#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(68) CometFilter -Input [3]: [d_date_sk#33, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2000)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#33)) +(77) CometFilter +Input [3]: [d_date_sk#31, d_year#46, d_moy#47] +Condition : ((((isnotnull(d_year#46) AND isnotnull(d_moy#47)) AND (d_year#46 = 2000)) AND (d_moy#47 = 2)) AND isnotnull(d_date_sk#31)) -(69) CometProject -Input [3]: [d_date_sk#33, d_year#49, d_moy#50] -Arguments: [d_date_sk#33], [d_date_sk#33] +(78) CometProject +Input [3]: [d_date_sk#31, d_year#46, d_moy#47] +Arguments: [d_date_sk#31], [d_date_sk#31] -(70) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] +(79) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#31] -(71) BroadcastExchange -Input [1]: [d_date_sk#33] +(80) BroadcastExchange +Input [1]: [d_date_sk#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (85) ++- * ColumnarToRow (84) + +- CometProject (83) + +- CometFilter (82) + +- CometScan parquet spark_catalog.default.date_dim (81) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#51] +Output [3]: [d_date_sk#10, d_date#11, d_year#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#51] -Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +(82) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#48] +Condition : (d_year#48 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(74) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#51] +(83) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#48] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(75) ColumnarToRow [codegen id : 1] +(84) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(76) BroadcastExchange +(85) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (91) -+- Exchange (90) - +- * HashAggregate (89) - +- * HashAggregate (88) - +- Exchange (87) - +- * HashAggregate (86) - +- * Project (85) - +- * BroadcastHashJoin Inner BuildRight (84) - :- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * ColumnarToRow (79) - : : +- CometFilter (78) - : : +- CometScan parquet spark_catalog.default.store_sales (77) - : +- ReusedExchange (80) - +- ReusedExchange (83) +Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#29, [id=#30] +* ColumnarToRow (102) ++- CometHashAggregate (101) + +- CometColumnarExchange (100) + +- CometHashAggregate (99) + +- CometHashAggregate (98) + +- CometColumnarExchange (97) + +- RowToColumnar (96) + +- * HashAggregate (95) + +- * Project (94) + +- * BroadcastHashJoin Inner BuildRight (93) + :- * Project (91) + : +- * BroadcastHashJoin Inner BuildRight (90) + : :- * ColumnarToRow (88) + : : +- CometFilter (87) + : : +- CometScan parquet spark_catalog.default.store_sales (86) + : +- ReusedExchange (89) + +- ReusedExchange (92) (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] +Output [4]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#55), dynamicpruningexpression(ss_sold_date_sk#55 IN dynamicpruning#56)] +PartitionFilters: [isnotnull(ss_sold_date_sk#52), dynamicpruningexpression(ss_sold_date_sk#52 IN dynamicpruning#53)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(78) CometFilter -Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] -Condition : isnotnull(ss_customer_sk#52) +(87) CometFilter +Input [4]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52] +Condition : isnotnull(ss_customer_sk#49) -(79) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] +(88) ColumnarToRow [codegen id : 3] +Input [4]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52] -(80) ReusedExchange [Reuses operator id: 32] -Output [1]: [c_customer_sk#57] +(89) ReusedExchange [Reuses operator id: 35] +Output [1]: [c_customer_sk#54] -(81) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#52] -Right keys [1]: [c_customer_sk#57] +(90) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_customer_sk#49] +Right keys [1]: [c_customer_sk#54] Join type: Inner Join condition: None -(82) Project [codegen id : 3] -Output [4]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] -Input [5]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] +(91) Project [codegen id : 3] +Output [4]: [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54] +Input [5]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54] -(83) ReusedExchange [Reuses operator id: 96] -Output [1]: [d_date_sk#58] +(92) ReusedExchange [Reuses operator id: 107] +Output [1]: [d_date_sk#55] -(84) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#55] -Right keys [1]: [d_date_sk#58] +(93) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#52] +Right keys [1]: [d_date_sk#55] Join type: Inner Join condition: None -(85) Project [codegen id : 3] -Output [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] -Input [5]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57, d_date_sk#58] - -(86) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] -Keys [1]: [c_customer_sk#57] -Functions [1]: [partial_sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] -Aggregate Attributes [2]: [sum#59, isEmpty#60] -Results [3]: [c_customer_sk#57, sum#61, isEmpty#62] - -(87) Exchange -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -Arguments: hashpartitioning(c_customer_sk#57, 5), ENSURE_REQUIREMENTS, [plan_id=11] - -(88) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -Keys [1]: [c_customer_sk#57] -Functions [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63] -Results [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63 AS csales#64] - -(89) HashAggregate [codegen id : 4] -Input [1]: [csales#64] +(94) Project [codegen id : 3] +Output [3]: [ss_quantity#50, ss_sales_price#51, c_customer_sk#54] +Input [5]: [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54, d_date_sk#55] + +(95) HashAggregate [codegen id : 3] +Input [3]: [ss_quantity#50, ss_sales_price#51, c_customer_sk#54] +Keys [1]: [c_customer_sk#54] +Functions [1]: [partial_sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))] +Aggregate Attributes [2]: [sum#56, isEmpty#57] +Results [3]: [c_customer_sk#54, sum#58, isEmpty#59] + +(96) RowToColumnar +Input [3]: [c_customer_sk#54, sum#58, isEmpty#59] + +(97) CometColumnarExchange +Input [3]: [c_customer_sk#54, sum#58, isEmpty#59] +Arguments: hashpartitioning(c_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(98) CometHashAggregate +Input [3]: [c_customer_sk#54, sum#58, isEmpty#59] +Keys [1]: [c_customer_sk#54] +Functions [1]: [sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))] + +(99) CometHashAggregate +Input [1]: [csales#60] Keys: [] -Functions [1]: [partial_max(csales#64)] -Aggregate Attributes [1]: [max#65] -Results [1]: [max#66] +Functions [1]: [partial_max(csales#60)] -(90) Exchange -Input [1]: [max#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(100) CometColumnarExchange +Input [1]: [max#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(91) HashAggregate [codegen id : 5] -Input [1]: [max#66] +(101) CometHashAggregate +Input [1]: [max#61] Keys: [] -Functions [1]: [max(csales#64)] -Aggregate Attributes [1]: [max(csales#64)#67] -Results [1]: [max(csales#64)#67 AS tpcds_cmax#68] +Functions [1]: [max(csales#60)] + +(102) ColumnarToRow [codegen id : 4] +Input [1]: [tpcds_cmax#62] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#55 IN dynamicpruning#56 -BroadcastExchange (96) -+- * ColumnarToRow (95) - +- CometProject (94) - +- CometFilter (93) - +- CometScan parquet spark_catalog.default.date_dim (92) +Subquery:4 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#52 IN dynamicpruning#53 +BroadcastExchange (107) ++- * ColumnarToRow (106) + +- CometProject (105) + +- CometFilter (104) + +- CometScan parquet spark_catalog.default.date_dim (103) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#58, d_year#69] +Output [2]: [d_date_sk#55, d_year#63] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(93) CometFilter -Input [2]: [d_date_sk#58, d_year#69] -Condition : (d_year#69 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#58)) +(104) CometFilter +Input [2]: [d_date_sk#55, d_year#63] +Condition : (d_year#63 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#55)) -(94) CometProject -Input [2]: [d_date_sk#58, d_year#69] -Arguments: [d_date_sk#58], [d_date_sk#58] +(105) CometProject +Input [2]: [d_date_sk#55, d_year#63] +Arguments: [d_date_sk#55], [d_date_sk#55] -(95) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#58] +(106) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#55] -(96) BroadcastExchange -Input [1]: [d_date_sk#58] +(107) BroadcastExchange +Input [1]: [d_date_sk#55] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:5 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:6 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index 0ec56d0e72..51a411e3c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -1,155 +1,154 @@ -WholeStageCodegen (24) - HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] +WholeStageCodegen (20) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (23) - HashAggregate [sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (11) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + CometHashAggregate [sum,isEmpty] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (19) + HashAggregate [sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (9) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] + CometSortMergeJoin [cs_bill_customer_sk,c_customer_sk] + CometSort [cs_bill_customer_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + RowToColumnar + WholeStageCodegen (5) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) + CometProject [item_sk] + CometFilter [cnt] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [ssales] + Subquery #3 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [max] + CometColumnarExchange #10 + CometHashAggregate [csales] + CometHashAggregate [c_customer_sk,sum,isEmpty] + CometColumnarExchange [c_customer_sk] #11 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - WholeStageCodegen (9) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - Exchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - InputAdapter - Exchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [c_customer_sk] #9 InputAdapter - ReusedExchange [c_customer_sk] #9 - InputAdapter - ReusedExchange [d_date_sk] #12 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] - InputAdapter - Exchange [c_customer_sk] #8 - WholeStageCodegen (8) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + ReusedExchange [d_date_sk] #12 + CometHashAggregate [c_customer_sk,sum,isEmpty] + CometColumnarExchange [c_customer_sk] #8 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter - WholeStageCodegen (17) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #13 - WholeStageCodegen (16) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (18) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] + CometSortMergeJoin [ws_bill_customer_sk,c_customer_sk] + CometSort [ws_bill_customer_sk] + CometColumnarExchange [ws_bill_customer_sk] #13 + RowToColumnar + WholeStageCodegen (14) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [c_customer_sk,sum,isEmpty] + ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter - WholeStageCodegen (20) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 86227b8324..5d9f6a6715 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -1,91 +1,101 @@ == Physical Plan == -TakeOrderedAndProject (87) -+- Union (86) - :- * HashAggregate (62) - : +- Exchange (61) - : +- * HashAggregate (60) - : +- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (56) - : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * SortMergeJoin LeftSemi (42) - : : : :- * Sort (25) - : : : : +- Exchange (24) - : : : : +- * Project (23) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (21) - : : : : +- * Project (20) - : : : : +- * Filter (19) - : : : : +- * HashAggregate (18) - : : : : +- Exchange (17) - : : : : +- * HashAggregate (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (7) - : : : : +- BroadcastExchange (13) - : : : : +- * ColumnarToRow (12) - : : : : +- CometFilter (11) - : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : +- * Sort (41) - : : : +- * Project (40) - : : : +- * Filter (39) - : : : +- * HashAggregate (38) - : : : +- Exchange (37) - : : : +- * HashAggregate (36) - : : : +- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (29) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan parquet spark_catalog.default.store_sales (26) - : : : +- BroadcastExchange (33) - : : : +- * ColumnarToRow (32) - : : : +- CometFilter (31) - : : : +- CometScan parquet spark_catalog.default.customer (30) - : : +- BroadcastExchange (54) - : : +- * SortMergeJoin LeftSemi (53) - : : :- * ColumnarToRow (47) - : : : +- CometSort (46) - : : : +- CometExchange (45) - : : : +- CometFilter (44) - : : : +- CometScan parquet spark_catalog.default.customer (43) - : : +- * Sort (52) - : : +- * Project (51) - : : +- * Filter (50) - : : +- * HashAggregate (49) - : : +- ReusedExchange (48) - : +- ReusedExchange (57) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * BroadcastHashJoin Inner BuildRight (81) - :- * Project (79) - : +- * BroadcastHashJoin Inner BuildRight (78) - : :- * SortMergeJoin LeftSemi (76) - : : :- * Sort (70) - : : : +- Exchange (69) - : : : +- * Project (68) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (67) - : : : :- * ColumnarToRow (65) - : : : : +- CometFilter (64) - : : : : +- CometScan parquet spark_catalog.default.web_sales (63) - : : : +- ReusedExchange (66) - : : +- * Sort (75) - : : +- * Project (74) - : : +- * Filter (73) - : : +- * HashAggregate (72) - : : +- ReusedExchange (71) - : +- ReusedExchange (77) - +- ReusedExchange (80) +* ColumnarToRow (97) ++- CometTakeOrderedAndProject (96) + +- CometUnion (95) + :- CometHashAggregate (68) + : +- CometColumnarExchange (67) + : +- RowToColumnar (66) + : +- * HashAggregate (65) + : +- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * ColumnarToRow (47) + : : : +- CometSortMergeJoin (46) + : : : :- CometSort (28) + : : : : +- CometColumnarExchange (27) + : : : : +- RowToColumnar (26) + : : : : +- * Project (25) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (24) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (23) + : : : : +- * ColumnarToRow (22) + : : : : +- CometProject (21) + : : : : +- CometFilter (20) + : : : : +- CometHashAggregate (19) + : : : : +- CometColumnarExchange (18) + : : : : +- RowToColumnar (17) + : : : : +- * HashAggregate (16) + : : : : +- * Project (15) + : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : :- * Project (9) + : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : :- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : : +- ReusedExchange (7) + : : : : +- BroadcastExchange (13) + : : : : +- * ColumnarToRow (12) + : : : : +- CometFilter (11) + : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : +- CometSort (45) + : : : +- CometProject (44) + : : : +- CometFilter (43) + : : : +- CometHashAggregate (42) + : : : +- CometColumnarExchange (41) + : : : +- RowToColumnar (40) + : : : +- * HashAggregate (39) + : : : +- * Project (38) + : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : :- * ColumnarToRow (32) + : : : : +- CometProject (31) + : : : : +- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.store_sales (29) + : : : +- BroadcastExchange (36) + : : : +- * ColumnarToRow (35) + : : : +- CometFilter (34) + : : : +- CometScan parquet spark_catalog.default.customer (33) + : : +- BroadcastExchange (59) + : : +- * ColumnarToRow (58) + : : +- CometSortMergeJoin (57) + : : :- CometSort (51) + : : : +- CometColumnarExchange (50) + : : : +- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.customer (48) + : : +- CometSort (56) + : : +- CometProject (55) + : : +- CometFilter (54) + : : +- CometHashAggregate (53) + : : +- ReusedExchange (52) + : +- ReusedExchange (62) + +- CometHashAggregate (94) + +- CometColumnarExchange (93) + +- RowToColumnar (92) + +- * HashAggregate (91) + +- * Project (90) + +- * BroadcastHashJoin Inner BuildRight (89) + :- * Project (87) + : +- * BroadcastHashJoin Inner BuildRight (86) + : :- * ColumnarToRow (84) + : : +- CometSortMergeJoin (83) + : : :- CometSort (77) + : : : +- CometColumnarExchange (76) + : : : +- RowToColumnar (75) + : : : +- * Project (74) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (73) + : : : :- * ColumnarToRow (71) + : : : : +- CometFilter (70) + : : : : +- CometScan parquet spark_catalog.default.web_sales (69) + : : : +- ReusedExchange (72) + : : +- CometSort (82) + : : +- CometProject (81) + : : +- CometFilter (80) + : : +- CometHashAggregate (79) + : : +- ReusedExchange (78) + : +- ReusedExchange (85) + +- ReusedExchange (88) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -118,7 +128,7 @@ Condition : isnotnull(ss_item_sk#7) (6) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(7) ReusedExchange [Reuses operator id: 97] +(7) ReusedExchange [Reuses operator id: 107] Output [2]: [d_date_sk#10, d_date#11] (8) BroadcastHashJoin [codegen id : 3] @@ -166,529 +176,548 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(17) Exchange +(17) RowToColumnar Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(18) HashAggregate [codegen id : 4] +(18) CometColumnarExchange +Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] +Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(19) CometHashAggregate Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(19) Filter [codegen id : 4] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) +(20) CometFilter +Input [2]: [item_sk#17, cnt#18] +Condition : (cnt#18 > 4) + +(21) CometProject +Input [2]: [item_sk#17, cnt#18] +Arguments: [item_sk#17], [item_sk#17] -(20) Project [codegen id : 4] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] +(22) ColumnarToRow [codegen id : 4] +Input [1]: [item_sk#17] -(21) BroadcastExchange -Input [1]: [item_sk#18] +(23) BroadcastExchange +Input [1]: [item_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 5] +(24) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] +Right keys [1]: [item_sk#17] Join type: LeftSemi Join condition: None -(23) Project [codegen id : 5] +(25) Project [codegen id : 5] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(24) Exchange +(26) RowToColumnar +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + +(27) CometColumnarExchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) Sort [codegen id : 6] +(28) CometSort Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 +Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(27) CometFilter -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#20) +(30) CometFilter +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#19) -(28) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] +(31) CometProject +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -(29) ColumnarToRow [codegen id : 8] -Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] +(32) ColumnarToRow [codegen id : 7] +Input [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] (unknown) Scan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] +Output [1]: [c_customer_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(31) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) +(34) CometFilter +Input [1]: [c_customer_sk#23] +Condition : isnotnull(c_customer_sk#23) -(32) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#24] +(35) ColumnarToRow [codegen id : 6] +Input [1]: [c_customer_sk#23] -(33) BroadcastExchange -Input [1]: [c_customer_sk#24] +(36) BroadcastExchange +Input [1]: [c_customer_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#20] -Right keys [1]: [c_customer_sk#24] +(37) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ss_customer_sk#19] +Right keys [1]: [c_customer_sk#23] Join type: Inner Join condition: None -(35) Project [codegen id : 8] -Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] - -(36) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Keys [1]: [c_customer_sk#24] -Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [2]: [sum#25, isEmpty#26] -Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] - -(37) Exchange -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(38) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] - -(39) Filter [codegen id : 9] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) - -(40) Project [codegen id : 9] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] - -(41) Sort [codegen id : 9] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(42) SortMergeJoin [codegen id : 16] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#24] -Join type: LeftSemi -Join condition: None +(38) Project [codegen id : 7] +Output [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] + +(39) HashAggregate [codegen id : 7] +Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +Keys [1]: [c_customer_sk#23] +Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] +Aggregate Attributes [2]: [sum#24, isEmpty#25] +Results [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(40) RowToColumnar +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(41) CometColumnarExchange +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(42) CometHashAggregate +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] + +(43) CometFilter +Input [2]: [c_customer_sk#23, ssales#28] +Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#29, [id=#30]))) + +(44) CometProject +Input [2]: [c_customer_sk#23, ssales#28] +Arguments: [c_customer_sk#23], [c_customer_sk#23] + +(45) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(46) CometSortMergeJoin +Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Right output [1]: [c_customer_sk#23] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi + +(47) ColumnarToRow [codegen id : 12] +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (unknown) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +Output [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(44) CometFilter -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Condition : isnotnull(c_customer_sk#33) +(49) CometFilter +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Condition : isnotnull(c_customer_sk#31) -(45) CometExchange -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: hashpartitioning(c_customer_sk#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(50) CometColumnarExchange +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(46) CometSort -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#33, c_first_name#34, c_last_name#35], [c_customer_sk#33 ASC NULLS FIRST] +(51) CometSort +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Arguments: [c_customer_sk#31, c_first_name#32, c_last_name#33], [c_customer_sk#31 ASC NULLS FIRST] -(47) ColumnarToRow [codegen id : 10] -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +(52) ReusedExchange [Reuses operator id: 41] +Output [3]: [c_customer_sk#23, sum#26, isEmpty#27] -(48) ReusedExchange [Reuses operator id: 37] -Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(53) CometHashAggregate +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] -(49) HashAggregate [codegen id : 13] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] +(54) CometFilter +Input [2]: [c_customer_sk#23, ssales#28] +Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30]))) -(50) Filter [codegen id : 13] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) +(55) CometProject +Input [2]: [c_customer_sk#23, ssales#28] +Arguments: [c_customer_sk#23], [c_customer_sk#23] -(51) Project [codegen id : 13] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] +(56) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] -(52) Sort [codegen id : 13] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 +(57) CometSortMergeJoin +Left output [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Right output [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#31], [c_customer_sk#23], LeftSemi -(53) SortMergeJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#33] -Right keys [1]: [c_customer_sk#24] -Join type: LeftSemi -Join condition: None +(58) ColumnarToRow [codegen id : 10] +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] -(54) BroadcastExchange -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +(59) BroadcastExchange +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(55) BroadcastHashJoin [codegen id : 16] +(60) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#33] +Right keys [1]: [c_customer_sk#31] Join type: Inner Join condition: None -(56) Project [codegen id : 16] -Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35] -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#33, c_first_name#34, c_last_name#35] +(61) Project [codegen id : 12] +Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#31, c_first_name#32, c_last_name#33] -(57) ReusedExchange [Reuses operator id: 92] -Output [1]: [d_date_sk#36] +(62) ReusedExchange [Reuses operator id: 102] +Output [1]: [d_date_sk#34] -(58) BroadcastHashJoin [codegen id : 16] +(63) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#36] +Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(59) Project [codegen id : 16] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35, d_date_sk#36] +(64) Project [codegen id : 12] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33, d_date_sk#34] -(60) HashAggregate [codegen id : 16] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] -Keys [2]: [c_last_name#35, c_first_name#34] +(65) HashAggregate [codegen id : 12] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] +Keys [2]: [c_last_name#33, c_first_name#32] Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] -Aggregate Attributes [2]: [sum#37, isEmpty#38] -Results [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] +Aggregate Attributes [2]: [sum#35, isEmpty#36] +Results [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -(61) Exchange -Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(66) RowToColumnar +Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -(62) HashAggregate [codegen id : 17] -Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -Keys [2]: [c_last_name#35, c_first_name#34] +(67) CometColumnarExchange +Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] +Arguments: hashpartitioning(c_last_name#33, c_first_name#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(68) CometHashAggregate +Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] +Keys [2]: [c_last_name#33, c_first_name#32] Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] -Aggregate Attributes [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41] -Results [3]: [c_last_name#35, c_first_name#34, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41 AS sales#42] (unknown) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] +PartitionFilters: [isnotnull(ws_sold_date_sk#43), dynamicpruningexpression(ws_sold_date_sk#43 IN dynamicpruning#44)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) CometFilter -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_customer_sk#44) +(70) CometFilter +Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] +Condition : isnotnull(ws_bill_customer_sk#40) -(65) ColumnarToRow [codegen id : 22] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +(71) ColumnarToRow [codegen id : 17] +Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -(66) ReusedExchange [Reuses operator id: 21] -Output [1]: [item_sk#18] +(72) ReusedExchange [Reuses operator id: 23] +Output [1]: [item_sk#17] -(67) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_item_sk#43] -Right keys [1]: [item_sk#18] +(73) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [item_sk#17] Join type: LeftSemi Join condition: None -(68) Project [codegen id : 22] -Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +(74) Project [codegen id : 17] +Output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] +Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -(69) Exchange -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(75) RowToColumnar +Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -(70) Sort [codegen id : 23] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 +(76) CometColumnarExchange +Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] +Arguments: hashpartitioning(ws_bill_customer_sk#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(71) ReusedExchange [Reuses operator id: 37] -Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(77) CometSort +Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] +Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40 ASC NULLS FIRST] -(72) HashAggregate [codegen id : 26] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] +(78) ReusedExchange [Reuses operator id: 41] +Output [3]: [c_customer_sk#23, sum#26, isEmpty#27] -(73) Filter [codegen id : 26] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) +(79) CometHashAggregate +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +Keys [1]: [c_customer_sk#23] +Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] -(74) Project [codegen id : 26] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] +(80) CometFilter +Input [2]: [c_customer_sk#23, ssales#28] +Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30]))) -(75) Sort [codegen id : 26] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 +(81) CometProject +Input [2]: [c_customer_sk#23, ssales#28] +Arguments: [c_customer_sk#23], [c_customer_sk#23] -(76) SortMergeJoin [codegen id : 33] -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#24] -Join type: LeftSemi -Join condition: None +(82) CometSort +Input [1]: [c_customer_sk#23] +Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] + +(83) CometSortMergeJoin +Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] +Right output [1]: [c_customer_sk#23] +Arguments: [ws_bill_customer_sk#40], [c_customer_sk#23], LeftSemi -(77) ReusedExchange [Reuses operator id: 54] -Output [3]: [c_customer_sk#49, c_first_name#50, c_last_name#51] +(84) ColumnarToRow [codegen id : 24] +Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -(78) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#49] +(85) ReusedExchange [Reuses operator id: 59] +Output [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] + +(86) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ws_bill_customer_sk#40] +Right keys [1]: [c_customer_sk#45] Join type: Inner Join condition: None -(79) Project [codegen id : 33] -Output [5]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#50, c_last_name#51] -Input [7]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_customer_sk#49, c_first_name#50, c_last_name#51] +(87) Project [codegen id : 24] +Output [5]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47] +Input [7]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_customer_sk#45, c_first_name#46, c_last_name#47] -(80) ReusedExchange [Reuses operator id: 92] -Output [1]: [d_date_sk#52] +(88) ReusedExchange [Reuses operator id: 102] +Output [1]: [d_date_sk#48] -(81) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#52] +(89) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ws_sold_date_sk#43] +Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(82) Project [codegen id : 33] -Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] -Input [6]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#50, c_last_name#51, d_date_sk#52] +(90) Project [codegen id : 24] +Output [4]: [ws_quantity#41, ws_list_price#42, c_first_name#46, c_last_name#47] +Input [6]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47, d_date_sk#48] + +(91) HashAggregate [codegen id : 24] +Input [4]: [ws_quantity#41, ws_list_price#42, c_first_name#46, c_last_name#47] +Keys [2]: [c_last_name#47, c_first_name#46] +Functions [1]: [partial_sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [4]: [c_last_name#47, c_first_name#46, sum#51, isEmpty#52] -(83) HashAggregate [codegen id : 33] -Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] -Keys [2]: [c_last_name#51, c_first_name#50] -Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] -Aggregate Attributes [2]: [sum#53, isEmpty#54] -Results [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] +(92) RowToColumnar +Input [4]: [c_last_name#47, c_first_name#46, sum#51, isEmpty#52] -(84) Exchange -Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] -Arguments: hashpartitioning(c_last_name#51, c_first_name#50, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(93) CometColumnarExchange +Input [4]: [c_last_name#47, c_first_name#46, sum#51, isEmpty#52] +Arguments: hashpartitioning(c_last_name#47, c_first_name#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(85) HashAggregate [codegen id : 34] -Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] -Keys [2]: [c_last_name#51, c_first_name#50] -Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] -Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#57] -Results [3]: [c_last_name#51, c_first_name#50, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#57 AS sales#58] +(94) CometHashAggregate +Input [4]: [c_last_name#47, c_first_name#46, sum#51, isEmpty#52] +Keys [2]: [c_last_name#47, c_first_name#46] +Functions [1]: [sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] -(86) Union +(95) CometUnion +Child 0 Input [3]: [c_last_name#33, c_first_name#32, sales#53] +Child 1 Input [3]: [c_last_name#47, c_first_name#46, sales#54] -(87) TakeOrderedAndProject -Input [3]: [c_last_name#35, c_first_name#34, sales#42] -Arguments: 100, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, sales#42 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, sales#42] +(96) CometTakeOrderedAndProject +Input [3]: [c_last_name#33, c_first_name#32, sales#53] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#33 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,sales#53 ASC NULLS FIRST], output=[c_last_name#33,c_first_name#32,sales#53]), 100, [c_last_name#33 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, sales#53 ASC NULLS FIRST], [c_last_name#33, c_first_name#32, sales#53] + +(97) ColumnarToRow [codegen id : 25] +Input [3]: [c_last_name#33, c_first_name#32, sales#53] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (92) -+- * ColumnarToRow (91) - +- CometProject (90) - +- CometFilter (89) - +- CometScan parquet spark_catalog.default.date_dim (88) +BroadcastExchange (102) ++- * ColumnarToRow (101) + +- CometProject (100) + +- CometFilter (99) + +- CometScan parquet spark_catalog.default.date_dim (98) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#36, d_year#59, d_moy#60] +Output [3]: [d_date_sk#34, d_year#55, d_moy#56] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(89) CometFilter -Input [3]: [d_date_sk#36, d_year#59, d_moy#60] -Condition : ((((isnotnull(d_year#59) AND isnotnull(d_moy#60)) AND (d_year#59 = 2000)) AND (d_moy#60 = 2)) AND isnotnull(d_date_sk#36)) +(99) CometFilter +Input [3]: [d_date_sk#34, d_year#55, d_moy#56] +Condition : ((((isnotnull(d_year#55) AND isnotnull(d_moy#56)) AND (d_year#55 = 2000)) AND (d_moy#56 = 2)) AND isnotnull(d_date_sk#34)) -(90) CometProject -Input [3]: [d_date_sk#36, d_year#59, d_moy#60] -Arguments: [d_date_sk#36], [d_date_sk#36] +(100) CometProject +Input [3]: [d_date_sk#34, d_year#55, d_moy#56] +Arguments: [d_date_sk#34], [d_date_sk#34] -(91) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#36] +(101) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#34] -(92) BroadcastExchange -Input [1]: [d_date_sk#36] +(102) BroadcastExchange +Input [1]: [d_date_sk#34] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (97) -+- * ColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometScan parquet spark_catalog.default.date_dim (93) +BroadcastExchange (107) ++- * ColumnarToRow (106) + +- CometProject (105) + +- CometFilter (104) + +- CometScan parquet spark_catalog.default.date_dim (103) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#61] +Output [3]: [d_date_sk#10, d_date#11, d_year#57] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(94) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#61] -Condition : (d_year#61 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +(104) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#57] +Condition : (d_year#57 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(95) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#61] +(105) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#57] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(96) ColumnarToRow [codegen id : 1] +(106) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(97) BroadcastExchange +(107) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (112) -+- Exchange (111) - +- * HashAggregate (110) - +- * HashAggregate (109) - +- Exchange (108) - +- * HashAggregate (107) - +- * Project (106) - +- * BroadcastHashJoin Inner BuildRight (105) - :- * Project (103) - : +- * BroadcastHashJoin Inner BuildRight (102) - : :- * ColumnarToRow (100) - : : +- CometFilter (99) - : : +- CometScan parquet spark_catalog.default.store_sales (98) - : +- ReusedExchange (101) - +- ReusedExchange (104) +Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#29, [id=#30] +* ColumnarToRow (124) ++- CometHashAggregate (123) + +- CometColumnarExchange (122) + +- CometHashAggregate (121) + +- CometHashAggregate (120) + +- CometColumnarExchange (119) + +- RowToColumnar (118) + +- * HashAggregate (117) + +- * Project (116) + +- * BroadcastHashJoin Inner BuildRight (115) + :- * Project (113) + : +- * BroadcastHashJoin Inner BuildRight (112) + : :- * ColumnarToRow (110) + : : +- CometFilter (109) + : : +- CometScan parquet spark_catalog.default.store_sales (108) + : +- ReusedExchange (111) + +- ReusedExchange (114) (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] +Output [4]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#65), dynamicpruningexpression(ss_sold_date_sk#65 IN dynamicpruning#66)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#62)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(99) CometFilter -Input [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] -Condition : isnotnull(ss_customer_sk#62) +(109) CometFilter +Input [4]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61] +Condition : isnotnull(ss_customer_sk#58) -(100) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] +(110) ColumnarToRow [codegen id : 3] +Input [4]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61] -(101) ReusedExchange [Reuses operator id: 33] -Output [1]: [c_customer_sk#67] +(111) ReusedExchange [Reuses operator id: 36] +Output [1]: [c_customer_sk#63] -(102) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#62] -Right keys [1]: [c_customer_sk#67] +(112) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_customer_sk#58] +Right keys [1]: [c_customer_sk#63] Join type: Inner Join condition: None -(103) Project [codegen id : 3] -Output [4]: [ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67] -Input [5]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67] +(113) Project [codegen id : 3] +Output [4]: [ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61, c_customer_sk#63] +Input [5]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61, c_customer_sk#63] -(104) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#68] +(114) ReusedExchange [Reuses operator id: 129] +Output [1]: [d_date_sk#64] -(105) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#65] -Right keys [1]: [d_date_sk#68] +(115) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(106) Project [codegen id : 3] -Output [3]: [ss_quantity#63, ss_sales_price#64, c_customer_sk#67] -Input [5]: [ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67, d_date_sk#68] - -(107) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#63, ss_sales_price#64, c_customer_sk#67] -Keys [1]: [c_customer_sk#67] -Functions [1]: [partial_sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))] -Aggregate Attributes [2]: [sum#69, isEmpty#70] -Results [3]: [c_customer_sk#67, sum#71, isEmpty#72] - -(108) Exchange -Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] -Arguments: hashpartitioning(c_customer_sk#67, 5), ENSURE_REQUIREMENTS, [plan_id=14] - -(109) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] -Keys [1]: [c_customer_sk#67] -Functions [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))#73] -Results [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))#73 AS csales#74] - -(110) HashAggregate [codegen id : 4] -Input [1]: [csales#74] +(116) Project [codegen id : 3] +Output [3]: [ss_quantity#59, ss_sales_price#60, c_customer_sk#63] +Input [5]: [ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61, c_customer_sk#63, d_date_sk#64] + +(117) HashAggregate [codegen id : 3] +Input [3]: [ss_quantity#59, ss_sales_price#60, c_customer_sk#63] +Keys [1]: [c_customer_sk#63] +Functions [1]: [partial_sum((cast(ss_quantity#59 as decimal(10,0)) * ss_sales_price#60))] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [3]: [c_customer_sk#63, sum#67, isEmpty#68] + +(118) RowToColumnar +Input [3]: [c_customer_sk#63, sum#67, isEmpty#68] + +(119) CometColumnarExchange +Input [3]: [c_customer_sk#63, sum#67, isEmpty#68] +Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(120) CometHashAggregate +Input [3]: [c_customer_sk#63, sum#67, isEmpty#68] +Keys [1]: [c_customer_sk#63] +Functions [1]: [sum((cast(ss_quantity#59 as decimal(10,0)) * ss_sales_price#60))] + +(121) CometHashAggregate +Input [1]: [csales#69] Keys: [] -Functions [1]: [partial_max(csales#74)] -Aggregate Attributes [1]: [max#75] -Results [1]: [max#76] +Functions [1]: [partial_max(csales#69)] -(111) Exchange -Input [1]: [max#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] +(122) CometColumnarExchange +Input [1]: [max#70] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(112) HashAggregate [codegen id : 5] -Input [1]: [max#76] +(123) CometHashAggregate +Input [1]: [max#70] Keys: [] -Functions [1]: [max(csales#74)] -Aggregate Attributes [1]: [max(csales#74)#77] -Results [1]: [max(csales#74)#77 AS tpcds_cmax#78] +Functions [1]: [max(csales#69)] + +(124) ColumnarToRow [codegen id : 4] +Input [1]: [tpcds_cmax#71] -Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#65 IN dynamicpruning#66 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +Subquery:4 Hosting operator id = 108 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#62 +BroadcastExchange (129) ++- * ColumnarToRow (128) + +- CometProject (127) + +- CometFilter (126) + +- CometScan parquet spark_catalog.default.date_dim (125) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#68, d_year#79] +Output [2]: [d_date_sk#64, d_year#72] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter -Input [2]: [d_date_sk#68, d_year#79] -Condition : (d_year#79 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#68)) +(126) CometFilter +Input [2]: [d_date_sk#64, d_year#72] +Condition : (d_year#72 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#64)) -(115) CometProject -Input [2]: [d_date_sk#68, d_year#79] -Arguments: [d_date_sk#68], [d_date_sk#68] +(127) CometProject +Input [2]: [d_date_sk#64, d_year#72] +Arguments: [d_date_sk#64], [d_date_sk#64] -(116) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#68] +(128) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#64] -(117) BroadcastExchange -Input [1]: [d_date_sk#68] +(129) BroadcastExchange +Input [1]: [d_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:5 Hosting operator id = 50 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] -Subquery:6 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 69 Hosting Expression = ws_sold_date_sk#43 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:7 Hosting operator id = 80 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 9cbb568a9a..2dd7ddf4af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -1,186 +1,179 @@ -TakeOrderedAndProject [c_last_name,c_first_name,sales] - Union - WholeStageCodegen (17) - HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (16) - HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] - Project [cs_quantity,cs_list_price,c_first_name,c_last_name] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (6) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] +WholeStageCodegen (25) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,sales] + CometUnion + CometHashAggregate [c_last_name,c_first_name,sum,isEmpty] + CometColumnarExchange [c_last_name,c_first_name] #1 + RowToColumnar + WholeStageCodegen (12) + HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] + Project [cs_quantity,cs_list_price,c_first_name,c_last_name] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometSortMergeJoin [cs_bill_customer_sk,c_customer_sk] + CometSort [cs_bill_customer_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + RowToColumnar + WholeStageCodegen (5) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) + CometProject [item_sk] + CometFilter [cnt] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [ssales] + Subquery #3 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [max] + CometColumnarExchange #10 + CometHashAggregate [csales] + CometHashAggregate [c_customer_sk,sum,isEmpty] + CometColumnarExchange [c_customer_sk] #11 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - WholeStageCodegen (9) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - Exchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - InputAdapter - Exchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [c_customer_sk] #9 InputAdapter - ReusedExchange [c_customer_sk] #9 - InputAdapter - ReusedExchange [d_date_sk] #12 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] - InputAdapter - Exchange [c_customer_sk] #8 - WholeStageCodegen (8) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (14) - SortMergeJoin [c_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (10) - ColumnarToRow - InputAdapter + ReusedExchange [d_date_sk] #12 + CometHashAggregate [c_customer_sk,sum,isEmpty] + CometColumnarExchange [c_customer_sk] #8 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometSortMergeJoin [c_customer_sk,c_customer_sk] CometSort [c_customer_sk] - CometExchange [c_customer_sk] #14 + CometColumnarExchange [c_customer_sk] #14 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - WholeStageCodegen (13) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (34) - HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name] #15 - WholeStageCodegen (33) - HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] - Project [ws_quantity,ws_list_price,c_first_name,c_last_name] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (23) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #16 - WholeStageCodegen (22) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - InputAdapter - WholeStageCodegen (26) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [c_customer_sk,sum,isEmpty] + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + InputAdapter + ReusedExchange [d_date_sk] #3 + CometHashAggregate [c_last_name,c_first_name,sum,isEmpty] + CometColumnarExchange [c_last_name,c_first_name] #15 + RowToColumnar + WholeStageCodegen (24) + HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] + Project [ws_quantity,ws_list_price,c_first_name,c_last_name] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometSortMergeJoin [ws_bill_customer_sk,c_customer_sk] + CometSort [ws_bill_customer_sk] + CometColumnarExchange [ws_bill_customer_sk] #16 + RowToColumnar + WholeStageCodegen (17) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [c_customer_sk,sum,isEmpty] + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index 6e03148d53..a7ccfef3ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -* Filter (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.item (21) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_address (33) +* ColumnarToRow (47) ++- CometFilter (46) + +- CometHashAggregate (45) + +- CometColumnarExchange (44) + +- CometHashAggregate (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * ColumnarToRow (13) + : : : : +- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.store (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.item (21) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_address (33) (unknown) Scan parquet spark_catalog.default.store_sales @@ -61,9 +63,9 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) CometExchange +(4) CometColumnarExchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] (5) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] @@ -84,9 +86,9 @@ Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(9) CometExchange +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] @@ -227,108 +229,110 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(40) Exchange +(40) RowToColumnar Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(41) HashAggregate [codegen id : 6] +(41) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(42) CometHashAggregate Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] -Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(42) HashAggregate [codegen id : 6] -Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] +(43) CometHashAggregate +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#30] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] -Functions [1]: [partial_sum(netpaid#31)] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Functions [1]: [partial_sum(netpaid#30)] -(43) Exchange -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(44) CometColumnarExchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#31, isEmpty#32] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(44) HashAggregate [codegen id : 7] -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +(45) CometHashAggregate +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#31, isEmpty#32] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] -Functions [1]: [sum(netpaid#31)] -Aggregate Attributes [1]: [sum(netpaid#31)#36] -Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] +Functions [1]: [sum(netpaid#30)] -(45) Filter [codegen id : 7] -Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] -Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) +(46) CometFilter +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#33] +Condition : (isnotnull(paid#33) AND (cast(paid#33 as decimal(33,8)) > cast(Subquery scalar-subquery#34, [id=#35] as decimal(33,8)))) + +(47) ColumnarToRow [codegen id : 6] +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#33] ===== Subqueries ===== -Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * ColumnarToRow (52) - : : : : +- CometProject (51) - : : : : +- CometSortMergeJoin (50) - : : : : :- CometSort (47) - : : : : : +- ReusedExchange (46) - : : : : +- CometSort (49) - : : : : +- ReusedExchange (48) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.item (56) - : +- ReusedExchange (62) - +- ReusedExchange (65) - - -(46) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#34, [id=#35] +* ColumnarToRow (77) ++- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- CometHashAggregate (74) + +- CometHashAggregate (73) + +- CometColumnarExchange (72) + +- RowToColumnar (71) + +- * HashAggregate (70) + +- * Project (69) + +- * BroadcastHashJoin Inner BuildRight (68) + :- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * Project (57) + : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : :- * ColumnarToRow (54) + : : : : +- CometProject (53) + : : : : +- CometSortMergeJoin (52) + : : : : :- CometSort (49) + : : : : : +- ReusedExchange (48) + : : : : +- CometSort (51) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (55) + : : +- BroadcastExchange (61) + : : +- * ColumnarToRow (60) + : : +- CometFilter (59) + : : +- CometScan parquet spark_catalog.default.item (58) + : +- ReusedExchange (64) + +- ReusedExchange (67) + + +(48) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(47) CometSort +(49) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(48) ReusedExchange [Reuses operator id: 9] +(50) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(49) CometSort +(51) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(50) CometSortMergeJoin +(52) CometSortMergeJoin Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Right output [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(51) CometProject +(53) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(52) ColumnarToRow [codegen id : 5] +(54) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(53) ReusedExchange [Reuses operator id: 18] +(55) ReusedExchange [Reuses operator id: 18] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(54) BroadcastHashJoin [codegen id : 5] +(56) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(55) Project [codegen id : 5] +(57) Project [codegen id : 5] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -339,87 +343,87 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(57) CometFilter +(59) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : isnotnull(i_item_sk#15) -(58) ColumnarToRow [codegen id : 2] +(60) ColumnarToRow [codegen id : 2] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(59) BroadcastExchange +(61) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 5] +(62) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(61) Project [codegen id : 5] +(63) Project [codegen id : 5] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(62) ReusedExchange [Reuses operator id: 30] +(64) ReusedExchange [Reuses operator id: 30] Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(63) BroadcastHashJoin [codegen id : 5] +(65) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(64) Project [codegen id : 5] +(66) Project [codegen id : 5] Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(65) ReusedExchange [Reuses operator id: 36] +(67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#25, ca_zip#26, ca_country#27] -(66) BroadcastHashJoin [codegen id : 5] +(68) BroadcastHashJoin [codegen id : 5] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(67) Project [codegen id : 5] +(69) Project [codegen id : 5] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(68) HashAggregate [codegen id : 5] +(70) HashAggregate [codegen id : 5] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#40] -Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] +Aggregate Attributes [1]: [sum#36] +Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] + +(71) RowToColumnar +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] -(69) Exchange -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(72) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(70) HashAggregate [codegen id : 6] -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] +(73) CometHashAggregate +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(71) HashAggregate [codegen id : 6] -Input [1]: [netpaid#31] +(74) CometHashAggregate +Input [1]: [netpaid#30] Keys: [] -Functions [1]: [partial_avg(netpaid#31)] -Aggregate Attributes [2]: [sum#42, count#43] -Results [2]: [sum#44, count#45] +Functions [1]: [partial_avg(netpaid#30)] -(72) Exchange -Input [2]: [sum#44, count#45] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] +(75) CometColumnarExchange +Input [2]: [sum#38, count#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(73) HashAggregate [codegen id : 7] -Input [2]: [sum#44, count#45] +(76) CometHashAggregate +Input [2]: [sum#38, count#39] Keys: [] -Functions [1]: [avg(netpaid#31)] -Aggregate Attributes [1]: [avg(netpaid#31)#46] -Results [1]: [(0.05 * avg(netpaid#31)#46) AS (0.05 * avg(netpaid))#47] +Functions [1]: [avg(netpaid#30)] + +(77) ColumnarToRow [codegen id : 6] +Input [1]: [(0.05 * avg(netpaid))#40] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index a4b8fef5d3..558a960ab1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -1,54 +1,54 @@ -WholeStageCodegen (7) - Filter [paid] - Subquery #1 - WholeStageCodegen (7) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #9 - WholeStageCodegen (6) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 +WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [paid] + Subquery #1 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [sum,count] + CometColumnarExchange #9 + CometHashAggregate [netpaid] + CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 + CometSort [sr_ticket_number,sr_item_sk] + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + BroadcastExchange #11 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 + ReusedExchange [ca_state,ca_zip,ca_country] #8 + CometHashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + CometHashAggregate [c_last_name,c_first_name,s_store_name,netpaid] + CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 + RowToColumnar WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] @@ -64,12 +64,12 @@ WholeStageCodegen (7) CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] CometSort [ss_ticket_number,ss_item_sk] - CometExchange [ss_ticket_number,ss_item_sk] #3 + CometColumnarExchange [ss_ticket_number,ss_item_sk] #3 CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometSort [sr_ticket_number,sr_item_sk] - CometExchange [sr_ticket_number,sr_item_sk] #4 + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt index fb3113df00..17f27af6ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -* Filter (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.item (21) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_address (33) +* ColumnarToRow (47) ++- CometFilter (46) + +- CometHashAggregate (45) + +- CometColumnarExchange (44) + +- CometHashAggregate (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * ColumnarToRow (13) + : : : : +- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.store (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.item (21) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_address (33) (unknown) Scan parquet spark_catalog.default.store_sales @@ -61,9 +63,9 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) CometExchange +(4) CometColumnarExchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] (5) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] @@ -84,9 +86,9 @@ Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(9) CometExchange +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] @@ -227,108 +229,110 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(40) Exchange +(40) RowToColumnar Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(41) HashAggregate [codegen id : 6] +(41) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(42) CometHashAggregate Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] -Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(42) HashAggregate [codegen id : 6] -Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] +(43) CometHashAggregate +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#30] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] -Functions [1]: [partial_sum(netpaid#31)] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Functions [1]: [partial_sum(netpaid#30)] -(43) Exchange -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(44) CometColumnarExchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#31, isEmpty#32] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(44) HashAggregate [codegen id : 7] -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +(45) CometHashAggregate +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#31, isEmpty#32] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] -Functions [1]: [sum(netpaid#31)] -Aggregate Attributes [1]: [sum(netpaid#31)#36] -Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] +Functions [1]: [sum(netpaid#30)] -(45) Filter [codegen id : 7] -Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] -Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) +(46) CometFilter +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#33] +Condition : (isnotnull(paid#33) AND (cast(paid#33 as decimal(33,8)) > cast(Subquery scalar-subquery#34, [id=#35] as decimal(33,8)))) + +(47) ColumnarToRow [codegen id : 6] +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#33] ===== Subqueries ===== -Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * ColumnarToRow (52) - : : : : +- CometProject (51) - : : : : +- CometSortMergeJoin (50) - : : : : :- CometSort (47) - : : : : : +- ReusedExchange (46) - : : : : +- CometSort (49) - : : : : +- ReusedExchange (48) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.item (56) - : +- ReusedExchange (62) - +- ReusedExchange (65) - - -(46) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#34, [id=#35] +* ColumnarToRow (77) ++- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- CometHashAggregate (74) + +- CometHashAggregate (73) + +- CometColumnarExchange (72) + +- RowToColumnar (71) + +- * HashAggregate (70) + +- * Project (69) + +- * BroadcastHashJoin Inner BuildRight (68) + :- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * Project (57) + : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : :- * ColumnarToRow (54) + : : : : +- CometProject (53) + : : : : +- CometSortMergeJoin (52) + : : : : :- CometSort (49) + : : : : : +- ReusedExchange (48) + : : : : +- CometSort (51) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (55) + : : +- BroadcastExchange (61) + : : +- * ColumnarToRow (60) + : : +- CometFilter (59) + : : +- CometScan parquet spark_catalog.default.item (58) + : +- ReusedExchange (64) + +- ReusedExchange (67) + + +(48) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(47) CometSort +(49) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(48) ReusedExchange [Reuses operator id: 9] +(50) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(49) CometSort +(51) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(50) CometSortMergeJoin +(52) CometSortMergeJoin Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Right output [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(51) CometProject +(53) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(52) ColumnarToRow [codegen id : 5] +(54) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(53) ReusedExchange [Reuses operator id: 18] +(55) ReusedExchange [Reuses operator id: 18] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(54) BroadcastHashJoin [codegen id : 5] +(56) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(55) Project [codegen id : 5] +(57) Project [codegen id : 5] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -339,87 +343,87 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(57) CometFilter +(59) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : isnotnull(i_item_sk#15) -(58) ColumnarToRow [codegen id : 2] +(60) ColumnarToRow [codegen id : 2] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(59) BroadcastExchange +(61) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 5] +(62) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(61) Project [codegen id : 5] +(63) Project [codegen id : 5] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(62) ReusedExchange [Reuses operator id: 30] +(64) ReusedExchange [Reuses operator id: 30] Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(63) BroadcastHashJoin [codegen id : 5] +(65) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(64) Project [codegen id : 5] +(66) Project [codegen id : 5] Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(65) ReusedExchange [Reuses operator id: 36] +(67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#25, ca_zip#26, ca_country#27] -(66) BroadcastHashJoin [codegen id : 5] +(68) BroadcastHashJoin [codegen id : 5] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(67) Project [codegen id : 5] +(69) Project [codegen id : 5] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(68) HashAggregate [codegen id : 5] +(70) HashAggregate [codegen id : 5] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#40] -Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] +Aggregate Attributes [1]: [sum#36] +Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] + +(71) RowToColumnar +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] -(69) Exchange -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(72) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(70) HashAggregate [codegen id : 6] -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] +(73) CometHashAggregate +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(71) HashAggregate [codegen id : 6] -Input [1]: [netpaid#31] +(74) CometHashAggregate +Input [1]: [netpaid#30] Keys: [] -Functions [1]: [partial_avg(netpaid#31)] -Aggregate Attributes [2]: [sum#42, count#43] -Results [2]: [sum#44, count#45] +Functions [1]: [partial_avg(netpaid#30)] -(72) Exchange -Input [2]: [sum#44, count#45] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] +(75) CometColumnarExchange +Input [2]: [sum#38, count#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(73) HashAggregate [codegen id : 7] -Input [2]: [sum#44, count#45] +(76) CometHashAggregate +Input [2]: [sum#38, count#39] Keys: [] -Functions [1]: [avg(netpaid#31)] -Aggregate Attributes [1]: [avg(netpaid#31)#46] -Results [1]: [(0.05 * avg(netpaid#31)#46) AS (0.05 * avg(netpaid))#47] +Functions [1]: [avg(netpaid#30)] + +(77) ColumnarToRow [codegen id : 6] +Input [1]: [(0.05 * avg(netpaid))#40] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index a4b8fef5d3..558a960ab1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -1,54 +1,54 @@ -WholeStageCodegen (7) - Filter [paid] - Subquery #1 - WholeStageCodegen (7) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #9 - WholeStageCodegen (6) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 +WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [paid] + Subquery #1 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [sum,count] + CometColumnarExchange #9 + CometHashAggregate [netpaid] + CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 + CometSort [sr_ticket_number,sr_item_sk] + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + BroadcastExchange #11 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 + ReusedExchange [ca_state,ca_zip,ca_country] #8 + CometHashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + CometHashAggregate [c_last_name,c_first_name,s_store_name,netpaid] + CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 + RowToColumnar WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] @@ -64,12 +64,12 @@ WholeStageCodegen (7) CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] CometSort [ss_ticket_number,ss_item_sk] - CometExchange [ss_ticket_number,ss_item_sk] #3 + CometColumnarExchange [ss_ticket_number,ss_item_sk] #3 CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometSort [sr_ticket_number,sr_item_sk] - CometExchange [sr_ticket_number,sr_item_sk] #4 + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt index 7eb30b6fb4..6b203e283f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt @@ -1,44 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +* ColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- RowToColumnar (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (18) + : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) + : : : : +- ReusedExchange (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.store (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.item (31) (unknown) Scan parquet spark_catalog.default.store_sales @@ -114,7 +116,7 @@ Join condition: None Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -(16) ReusedExchange [Reuses operator id: 45] +(16) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#19] (17) BroadcastHashJoin [codegen id : 8] @@ -127,7 +129,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] -(19) ReusedExchange [Reuses operator id: 50] +(19) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#20] (20) BroadcastHashJoin [codegen id : 8] @@ -140,7 +142,7 @@ Join condition: None Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#20] -(22) ReusedExchange [Reuses operator id: 50] +(22) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#21] (23) BroadcastHashJoin [codegen id : 8] @@ -216,80 +218,84 @@ Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(Unscale Aggregate Attributes [3]: [sum#28, sum#29, sum#30] Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -(38) Exchange +(38) RowToColumnar Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(39) HashAggregate [codegen id : 9] +(39) CometColumnarExchange +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(40) CometHashAggregate Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#34, sum(UnscaledValue(sr_net_loss#11))#35, sum(UnscaledValue(cs_net_profit#16))#36] -Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#34,17,2) AS store_sales_profit#37, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#35,17,2) AS store_returns_loss#38, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#36,17,2) AS catalog_sales_profit#39] -(40) TakeOrderedAndProject -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] -Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] +(41) CometTakeOrderedAndProject +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#34, store_returns_loss#35, catalog_sales_profit#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#26 ASC NULLS FIRST,i_item_desc#27 ASC NULLS FIRST,s_store_id#23 ASC NULLS FIRST,s_store_name#24 ASC NULLS FIRST], output=[i_item_id#26,i_item_desc#27,s_store_id#23,s_store_name#24,store_sales_profit#34,store_returns_loss#35,catalog_sales_profit#36]), 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#34, store_returns_loss#35, catalog_sales_profit#36] + +(42) ColumnarToRow [codegen id : 9] +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#34, store_returns_loss#35, catalog_sales_profit#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#40, d_moy#41] +Output [3]: [d_date_sk#19, d_year#37, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] -Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 4)) AND (d_year#40 = 2001)) AND isnotnull(d_date_sk#19)) +(44) CometFilter +Input [3]: [d_date_sk#19, d_year#37, d_moy#38] +Condition : ((((isnotnull(d_moy#38) AND isnotnull(d_year#37)) AND (d_moy#38 = 4)) AND (d_year#37 = 2001)) AND isnotnull(d_date_sk#19)) -(43) CometProject -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] +(45) CometProject +Input [3]: [d_date_sk#19, d_year#37, d_moy#38] Arguments: [d_date_sk#19], [d_date_sk#19] -(44) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(45) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#42, d_moy#43] +Output [3]: [d_date_sk#20, d_year#39, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] -Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 4)) AND (d_moy#43 <= 10)) AND (d_year#42 = 2001)) AND isnotnull(d_date_sk#20)) +(49) CometFilter +Input [3]: [d_date_sk#20, d_year#39, d_moy#40] +Condition : (((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 >= 4)) AND (d_moy#40 <= 10)) AND (d_year#39 = 2001)) AND isnotnull(d_date_sk#20)) -(48) CometProject -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] +(50) CometProject +Input [3]: [d_date_sk#20, d_year#39, d_moy#40] Arguments: [d_date_sk#20], [d_date_sk#20] -(49) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#20] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index eda7f6b64d..9d8f0b4415 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -1,76 +1,78 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] - Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] + CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] + Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #4 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #4 + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index 55bae9c8c3..cdb47fded7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -1,34 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan parquet spark_catalog.default.promotion (20) +* ColumnarToRow (32) ++- CometTakeOrderedAndProject (31) + +- CometHashAggregate (30) + +- CometColumnarExchange (29) + +- RowToColumnar (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * ColumnarToRow (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : +- ReusedExchange (11) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.item (14) + +- BroadcastExchange (24) + +- * ColumnarToRow (23) + +- CometProject (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.promotion (20) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -78,7 +80,7 @@ Join condition: None Output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] -(11) ReusedExchange [Reuses operator id: 35] +(11) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#14] (12) BroadcastHashJoin [codegen id : 5] @@ -158,50 +160,54 @@ Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_pr Aggregate Attributes [8]: [sum#20, count#21, sum#22, count#23, sum#24, count#25, sum#26, count#27] Results [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -(28) Exchange +(28) RowToColumnar Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) HashAggregate [codegen id : 6] +(29) CometColumnarExchange +Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometHashAggregate Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] Keys [1]: [i_item_id#16] Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [4]: [avg(cs_quantity#4)#36, avg(UnscaledValue(cs_list_price#5))#37, avg(UnscaledValue(cs_coupon_amt#7))#38, avg(UnscaledValue(cs_sales_price#6))#39] -Results [5]: [i_item_id#16, avg(cs_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(cs_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(cs_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(cs_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] -(30) TakeOrderedAndProject -Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] -Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] +(31) CometTakeOrderedAndProject +Input [5]: [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#16 ASC NULLS FIRST], output=[i_item_id#16,agg1#36,agg2#37,agg3#38,agg4#39]), 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] + +(32) ColumnarToRow [codegen id : 6] +Input [5]: [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#44] +Output [2]: [d_date_sk#14, d_year#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter -Input [2]: [d_date_sk#14, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) +(34) CometFilter +Input [2]: [d_date_sk#14, d_year#40] +Condition : ((isnotnull(d_year#40) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#14)) -(33) CometProject -Input [2]: [d_date_sk#14, d_year#44] +(35) CometProject +Input [2]: [d_date_sk#14, d_year#40] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(35) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index 7d38936244..1d9e320882 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -1,52 +1,54 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange [i_item_id] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometProject [p_promo_sk] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index eca71fb097..7a4a7aaf9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -1,34 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- * Expand (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.store (14) - +- BroadcastExchange (23) - +- * ColumnarToRow (22) - +- CometFilter (21) - +- CometScan parquet spark_catalog.default.item (20) +* ColumnarToRow (32) ++- CometTakeOrderedAndProject (31) + +- CometHashAggregate (30) + +- CometColumnarExchange (29) + +- RowToColumnar (28) + +- * HashAggregate (27) + +- * Expand (26) + +- * Project (25) + +- * BroadcastHashJoin Inner BuildRight (24) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * ColumnarToRow (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : +- ReusedExchange (11) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.store (14) + +- BroadcastExchange (23) + +- * ColumnarToRow (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.item (20) (unknown) Scan parquet spark_catalog.default.store_sales @@ -78,7 +80,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(11) ReusedExchange [Reuses operator id: 35] +(11) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#14] (12) BroadcastHashJoin [codegen id : 5] @@ -158,50 +160,54 @@ Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_pr Aggregate Attributes [8]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] Results [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] -(28) Exchange +(28) RowToColumnar Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] -Arguments: hashpartitioning(i_item_id#19, s_state#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) HashAggregate [codegen id : 6] +(29) CometColumnarExchange +Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] +Arguments: hashpartitioning(i_item_id#19, s_state#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometHashAggregate Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] Keys [3]: [i_item_id#19, s_state#20, spark_grouping_id#21] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#38, avg(UnscaledValue(ss_list_price#5))#39, avg(UnscaledValue(ss_coupon_amt#7))#40, avg(UnscaledValue(ss_sales_price#6))#41] -Results [7]: [i_item_id#19, s_state#20, cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) AS g_state#42, avg(ss_quantity#4)#38 AS agg1#43, cast((avg(UnscaledValue(ss_list_price#5))#39 / 100.0) as decimal(11,6)) AS agg2#44, cast((avg(UnscaledValue(ss_coupon_amt#7))#40 / 100.0) as decimal(11,6)) AS agg3#45, cast((avg(UnscaledValue(ss_sales_price#6))#41 / 100.0) as decimal(11,6)) AS agg4#46] -(30) TakeOrderedAndProject -Input [7]: [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] -Arguments: 100, [i_item_id#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST], [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] +(31) CometTakeOrderedAndProject +Input [7]: [i_item_id#19, s_state#20, g_state#38, agg1#39, agg2#40, agg3#41, agg4#42] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#19 ASC NULLS FIRST,s_state#20 ASC NULLS FIRST], output=[i_item_id#19,s_state#20,g_state#38,agg1#39,agg2#40,agg3#41,agg4#42]), 100, [i_item_id#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST], [i_item_id#19, s_state#20, g_state#38, agg1#39, agg2#40, agg3#41, agg4#42] + +(32) ColumnarToRow [codegen id : 6] +Input [7]: [i_item_id#19, s_state#20, g_state#38, agg1#39, agg2#40, agg3#41, agg4#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#47] +Output [2]: [d_date_sk#14, d_year#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter -Input [2]: [d_date_sk#14, d_year#47] -Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#14)) +(34) CometFilter +Input [2]: [d_date_sk#14, d_year#43] +Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2002)) AND isnotnull(d_date_sk#14)) -(33) CometProject -Input [2]: [d_date_sk#14, d_year#47] +(35) CometProject +Input [2]: [d_date_sk#14, d_year#43] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(35) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index 9d073ff67c..191ff0322c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -1,52 +1,54 @@ -TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id,s_state,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange [i_item_id,s_state,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt index db6d112e64..16063ddeac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt @@ -1,74 +1,86 @@ == Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (70) -:- * BroadcastNestedLoopJoin Inner BuildRight (58) -: :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (34) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (22) -: : : : :- * HashAggregate (10) -: : : : : +- Exchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- * ColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (21) -: : : : +- * HashAggregate (20) -: : : : +- Exchange (19) -: : : : +- * HashAggregate (18) -: : : : +- * HashAggregate (17) -: : : : +- * ColumnarToRow (16) -: : : : +- CometExchange (15) -: : : : +- CometHashAggregate (14) -: : : : +- CometProject (13) -: : : : +- CometFilter (12) -: : : : +- CometScan parquet spark_catalog.default.store_sales (11) -: : : +- BroadcastExchange (33) -: : : +- * HashAggregate (32) -: : : +- Exchange (31) -: : : +- * HashAggregate (30) -: : : +- * HashAggregate (29) -: : : +- * ColumnarToRow (28) -: : : +- CometExchange (27) -: : : +- CometHashAggregate (26) -: : : +- CometProject (25) -: : : +- CometFilter (24) -: : : +- CometScan parquet spark_catalog.default.store_sales (23) -: : +- BroadcastExchange (45) -: : +- * HashAggregate (44) -: : +- Exchange (43) -: : +- * HashAggregate (42) -: : +- * HashAggregate (41) -: : +- * ColumnarToRow (40) -: : +- CometExchange (39) -: : +- CometHashAggregate (38) -: : +- CometProject (37) -: : +- CometFilter (36) -: : +- CometScan parquet spark_catalog.default.store_sales (35) -: +- BroadcastExchange (57) -: +- * HashAggregate (56) -: +- Exchange (55) -: +- * HashAggregate (54) -: +- * HashAggregate (53) -: +- * ColumnarToRow (52) -: +- CometExchange (51) -: +- CometHashAggregate (50) -: +- CometProject (49) -: +- CometFilter (48) -: +- CometScan parquet spark_catalog.default.store_sales (47) -+- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * HashAggregate (65) - +- * ColumnarToRow (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.store_sales (59) +* BroadcastNestedLoopJoin Inner BuildRight (82) +:- * BroadcastNestedLoopJoin Inner BuildRight (68) +: :- * BroadcastNestedLoopJoin Inner BuildRight (54) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (40) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (26) +: : : : :- * ColumnarToRow (12) +: : : : : +- CometHashAggregate (11) +: : : : : +- CometColumnarExchange (10) +: : : : : +- RowToColumnar (9) +: : : : : +- * HashAggregate (8) +: : : : : +- * HashAggregate (7) +: : : : : +- * ColumnarToRow (6) +: : : : : +- CometColumnarExchange (5) +: : : : : +- CometHashAggregate (4) +: : : : : +- CometProject (3) +: : : : : +- CometFilter (2) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : +- BroadcastExchange (25) +: : : : +- * ColumnarToRow (24) +: : : : +- CometHashAggregate (23) +: : : : +- CometColumnarExchange (22) +: : : : +- RowToColumnar (21) +: : : : +- * HashAggregate (20) +: : : : +- * HashAggregate (19) +: : : : +- * ColumnarToRow (18) +: : : : +- CometColumnarExchange (17) +: : : : +- CometHashAggregate (16) +: : : : +- CometProject (15) +: : : : +- CometFilter (14) +: : : : +- CometScan parquet spark_catalog.default.store_sales (13) +: : : +- BroadcastExchange (39) +: : : +- * ColumnarToRow (38) +: : : +- CometHashAggregate (37) +: : : +- CometColumnarExchange (36) +: : : +- RowToColumnar (35) +: : : +- * HashAggregate (34) +: : : +- * HashAggregate (33) +: : : +- * ColumnarToRow (32) +: : : +- CometColumnarExchange (31) +: : : +- CometHashAggregate (30) +: : : +- CometProject (29) +: : : +- CometFilter (28) +: : : +- CometScan parquet spark_catalog.default.store_sales (27) +: : +- BroadcastExchange (53) +: : +- * ColumnarToRow (52) +: : +- CometHashAggregate (51) +: : +- CometColumnarExchange (50) +: : +- RowToColumnar (49) +: : +- * HashAggregate (48) +: : +- * HashAggregate (47) +: : +- * ColumnarToRow (46) +: : +- CometColumnarExchange (45) +: : +- CometHashAggregate (44) +: : +- CometProject (43) +: : +- CometFilter (42) +: : +- CometScan parquet spark_catalog.default.store_sales (41) +: +- BroadcastExchange (67) +: +- * ColumnarToRow (66) +: +- CometHashAggregate (65) +: +- CometColumnarExchange (64) +: +- RowToColumnar (63) +: +- * HashAggregate (62) +: +- * HashAggregate (61) +: +- * ColumnarToRow (60) +: +- CometColumnarExchange (59) +: +- CometHashAggregate (58) +: +- CometProject (57) +: +- CometFilter (56) +: +- CometScan parquet spark_catalog.default.store_sales (55) ++- BroadcastExchange (81) + +- * ColumnarToRow (80) + +- CometHashAggregate (79) + +- CometColumnarExchange (78) + +- RowToColumnar (77) + +- * HashAggregate (76) + +- * HashAggregate (75) + +- * ColumnarToRow (74) + +- CometColumnarExchange (73) + +- CometHashAggregate (72) + +- CometProject (71) + +- CometFilter (70) + +- CometScan parquet spark_catalog.default.store_sales (69) (unknown) Scan parquet spark_catalog.default.store_sales @@ -91,9 +103,9 @@ Input [1]: [ss_list_price#3] Keys [1]: [ss_list_price#3] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -(5) CometExchange +(5) CometColumnarExchange Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] (6) ColumnarToRow [codegen id : 1] Input [4]: [ss_list_price#3, sum#6, count#7, count#8] @@ -112,16 +124,20 @@ Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_p Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] Results [4]: [sum#6, count#7, count#8, count#12] -(9) Exchange +(9) RowToColumnar Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(10) HashAggregate [codegen id : 12] +(10) CometColumnarExchange +Input [4]: [sum#6, count#7, count#8, count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(11) CometHashAggregate Input [4]: [sum#6, count#7, count#8, count#12] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#9 / 100.0) as decimal(11,6)) AS B1_LP#13, count(ss_list_price#3)#10 AS B1_CNT#14, count(ss_list_price#3)#11 AS B1_CNTD#15] + +(12) ColumnarToRow [codegen id : 12] +Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] (unknown) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] @@ -130,56 +146,60 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct -(12) CometFilter +(14) CometFilter Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) -(13) CometProject +(15) CometProject Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Arguments: [ss_list_price#18], [ss_list_price#18] -(14) CometHashAggregate +(16) CometHashAggregate Input [1]: [ss_list_price#18] Keys [1]: [ss_list_price#18] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] -(15) CometExchange +(17) CometColumnarExchange Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(16) ColumnarToRow [codegen id : 2] +(18) ColumnarToRow [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -(17) HashAggregate [codegen id : 2] +(19) HashAggregate [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Keys [1]: [ss_list_price#18] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] Results [4]: [ss_list_price#18, sum#21, count#22, count#23] -(18) HashAggregate [codegen id : 2] +(20) HashAggregate [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] Results [4]: [sum#21, count#22, count#23, count#27] -(19) Exchange +(21) RowToColumnar Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(20) HashAggregate [codegen id : 3] +(22) CometColumnarExchange +Input [4]: [sum#21, count#22, count#23, count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(23) CometHashAggregate Input [4]: [sum#21, count#22, count#23, count#27] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#18))#24 / 100.0) as decimal(11,6)) AS B2_LP#28, count(ss_list_price#18)#25 AS B2_CNT#29, count(ss_list_price#18)#26 AS B2_CNTD#30] -(21) BroadcastExchange +(24) ColumnarToRow [codegen id : 3] +Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] + +(25) BroadcastExchange Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] Arguments: IdentityBroadcastMode, [plan_id=5] -(22) BroadcastNestedLoopJoin [codegen id : 12] +(26) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None @@ -190,56 +210,60 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct -(24) CometFilter +(28) CometFilter Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) -(25) CometProject +(29) CometProject Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Arguments: [ss_list_price#33], [ss_list_price#33] -(26) CometHashAggregate +(30) CometHashAggregate Input [1]: [ss_list_price#33] Keys [1]: [ss_list_price#33] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] -(27) CometExchange +(31) CometColumnarExchange Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(28) ColumnarToRow [codegen id : 4] +(32) ColumnarToRow [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -(29) HashAggregate [codegen id : 4] +(33) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys [1]: [ss_list_price#33] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] Results [4]: [ss_list_price#33, sum#36, count#37, count#38] -(30) HashAggregate [codegen id : 4] +(34) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] Results [4]: [sum#36, count#37, count#38, count#42] -(31) Exchange +(35) RowToColumnar +Input [4]: [sum#36, count#37, count#38, count#42] + +(36) CometColumnarExchange Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(32) HashAggregate [codegen id : 5] +(37) CometHashAggregate Input [4]: [sum#36, count#37, count#38, count#42] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#33))#39 / 100.0) as decimal(11,6)) AS B3_LP#43, count(ss_list_price#33)#40 AS B3_CNT#44, count(ss_list_price#33)#41 AS B3_CNTD#45] -(33) BroadcastExchange +(38) ColumnarToRow [codegen id : 5] +Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] + +(39) BroadcastExchange Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] Arguments: IdentityBroadcastMode, [plan_id=8] -(34) BroadcastNestedLoopJoin [codegen id : 12] +(40) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None @@ -250,56 +274,60 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct -(36) CometFilter +(42) CometFilter Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) -(37) CometProject +(43) CometProject Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Arguments: [ss_list_price#48], [ss_list_price#48] -(38) CometHashAggregate +(44) CometHashAggregate Input [1]: [ss_list_price#48] Keys [1]: [ss_list_price#48] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] -(39) CometExchange +(45) CometColumnarExchange Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] +Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(40) ColumnarToRow [codegen id : 6] +(46) ColumnarToRow [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -(41) HashAggregate [codegen id : 6] +(47) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys [1]: [ss_list_price#48] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] Results [4]: [ss_list_price#48, sum#51, count#52, count#53] -(42) HashAggregate [codegen id : 6] +(48) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] Results [4]: [sum#51, count#52, count#53, count#57] -(43) Exchange +(49) RowToColumnar Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(44) HashAggregate [codegen id : 7] +(50) CometColumnarExchange +Input [4]: [sum#51, count#52, count#53, count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(51) CometHashAggregate Input [4]: [sum#51, count#52, count#53, count#57] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#48))#54 / 100.0) as decimal(11,6)) AS B4_LP#58, count(ss_list_price#48)#55 AS B4_CNT#59, count(ss_list_price#48)#56 AS B4_CNTD#60] -(45) BroadcastExchange +(52) ColumnarToRow [codegen id : 7] +Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] + +(53) BroadcastExchange Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] Arguments: IdentityBroadcastMode, [plan_id=11] -(46) BroadcastNestedLoopJoin [codegen id : 12] +(54) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None @@ -310,56 +338,60 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct -(48) CometFilter +(56) CometFilter Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) -(49) CometProject +(57) CometProject Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Arguments: [ss_list_price#63], [ss_list_price#63] -(50) CometHashAggregate +(58) CometHashAggregate Input [1]: [ss_list_price#63] Keys [1]: [ss_list_price#63] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] -(51) CometExchange +(59) CometColumnarExchange Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(52) ColumnarToRow [codegen id : 8] +(60) ColumnarToRow [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -(53) HashAggregate [codegen id : 8] +(61) HashAggregate [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys [1]: [ss_list_price#63] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] Results [4]: [ss_list_price#63, sum#66, count#67, count#68] -(54) HashAggregate [codegen id : 8] +(62) HashAggregate [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] Results [4]: [sum#66, count#67, count#68, count#72] -(55) Exchange +(63) RowToColumnar Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(56) HashAggregate [codegen id : 9] +(64) CometColumnarExchange +Input [4]: [sum#66, count#67, count#68, count#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(65) CometHashAggregate Input [4]: [sum#66, count#67, count#68, count#72] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#63))#69 / 100.0) as decimal(11,6)) AS B5_LP#73, count(ss_list_price#63)#70 AS B5_CNT#74, count(ss_list_price#63)#71 AS B5_CNTD#75] -(57) BroadcastExchange +(66) ColumnarToRow [codegen id : 9] +Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] + +(67) BroadcastExchange Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] Arguments: IdentityBroadcastMode, [plan_id=14] -(58) BroadcastNestedLoopJoin [codegen id : 12] +(68) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None @@ -370,56 +402,60 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct -(60) CometFilter +(70) CometFilter Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) -(61) CometProject +(71) CometProject Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Arguments: [ss_list_price#78], [ss_list_price#78] -(62) CometHashAggregate +(72) CometHashAggregate Input [1]: [ss_list_price#78] Keys [1]: [ss_list_price#78] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] -(63) CometExchange +(73) CometColumnarExchange Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] +Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(64) ColumnarToRow [codegen id : 10] +(74) ColumnarToRow [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -(65) HashAggregate [codegen id : 10] +(75) HashAggregate [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys [1]: [ss_list_price#78] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] Results [4]: [ss_list_price#78, sum#81, count#82, count#83] -(66) HashAggregate [codegen id : 10] +(76) HashAggregate [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] Results [4]: [sum#81, count#82, count#83, count#87] -(67) Exchange +(77) RowToColumnar +Input [4]: [sum#81, count#82, count#83, count#87] + +(78) CometColumnarExchange Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(68) HashAggregate [codegen id : 11] +(79) CometHashAggregate Input [4]: [sum#81, count#82, count#83, count#87] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#78))#84 / 100.0) as decimal(11,6)) AS B6_LP#88, count(ss_list_price#78)#85 AS B6_CNT#89, count(ss_list_price#78)#86 AS B6_CNTD#90] -(69) BroadcastExchange +(80) ColumnarToRow [codegen id : 11] +Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] + +(81) BroadcastExchange Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] Arguments: IdentityBroadcastMode, [plan_id=17] -(70) BroadcastNestedLoopJoin [codegen id : 12] +(82) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt index 2f9802ab65..e84abd7c1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt @@ -4,96 +4,108 @@ WholeStageCodegen (12) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometExchange [ss_list_price] #2 - CometHashAggregate [ss_list_price] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometHashAggregate [sum,count,count,count] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (1) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #2 + CometHashAggregate [ss_list_price] + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count] + ColumnarToRow InputAdapter - Exchange #4 - WholeStageCodegen (2) + CometHashAggregate [sum,count,count,count] + CometColumnarExchange #4 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #5 + CometHashAggregate [ss_list_price] + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [sum,count,count,count] + CometColumnarExchange #7 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_list_price] #8 + CometHashAggregate [ss_list_price] + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [sum,count,count,count] + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (6) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] ColumnarToRow InputAdapter - CometExchange [ss_list_price] #5 + CometColumnarExchange [ss_list_price] #11 CometHashAggregate [ss_list_price] CometProject [ss_list_price] CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count] - InputAdapter - Exchange #7 - WholeStageCodegen (4) + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometHashAggregate [sum,count,count,count] + CometColumnarExchange #13 + RowToColumnar + WholeStageCodegen (8) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] ColumnarToRow InputAdapter - CometExchange [ss_list_price] #8 + CometColumnarExchange [ss_list_price] #14 CometHashAggregate [ss_list_price] CometProject [ss_list_price] CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count] - InputAdapter - Exchange #10 - WholeStageCodegen (6) + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometHashAggregate [sum,count,count,count] + CometColumnarExchange #16 + RowToColumnar + WholeStageCodegen (10) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] ColumnarToRow InputAdapter - CometExchange [ss_list_price] #11 + CometColumnarExchange [ss_list_price] #17 CometHashAggregate [ss_list_price] CometProject [ss_list_price] CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count] - InputAdapter - Exchange #13 - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 - CometHashAggregate [ss_list_price] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count] - InputAdapter - Exchange #16 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 - CometHashAggregate [ss_list_price] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt index e74f10d25a..69e36471f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt @@ -1,44 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +* ColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- RowToColumnar (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (18) + : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) + : : : : +- ReusedExchange (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.store (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.item (31) (unknown) Scan parquet spark_catalog.default.store_sales @@ -114,7 +116,7 @@ Join condition: None Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -(16) ReusedExchange [Reuses operator id: 45] +(16) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#19] (17) BroadcastHashJoin [codegen id : 8] @@ -127,7 +129,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -(19) ReusedExchange [Reuses operator id: 50] +(19) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#20] (20) BroadcastHashJoin [codegen id : 8] @@ -140,7 +142,7 @@ Join condition: None Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] -(22) ReusedExchange [Reuses operator id: 55] +(22) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#21] (23) BroadcastHashJoin [codegen id : 8] @@ -216,110 +218,114 @@ Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), Aggregate Attributes [3]: [sum#28, sum#29, sum#30] Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -(38) Exchange +(38) RowToColumnar Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(39) HashAggregate [codegen id : 9] +(39) CometColumnarExchange +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(40) CometHashAggregate Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] -Aggregate Attributes [3]: [sum(ss_quantity#5)#34, sum(sr_return_quantity#11)#35, sum(cs_quantity#16)#36] -Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum(ss_quantity#5)#34 AS store_sales_quantity#37, sum(sr_return_quantity#11)#35 AS store_returns_quantity#38, sum(cs_quantity#16)#36 AS catalog_sales_quantity#39] -(40) TakeOrderedAndProject -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] -Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] +(41) CometTakeOrderedAndProject +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#34, store_returns_quantity#35, catalog_sales_quantity#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#26 ASC NULLS FIRST,i_item_desc#27 ASC NULLS FIRST,s_store_id#23 ASC NULLS FIRST,s_store_name#24 ASC NULLS FIRST], output=[i_item_id#26,i_item_desc#27,s_store_id#23,s_store_name#24,store_sales_quantity#34,store_returns_quantity#35,catalog_sales_quantity#36]), 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#34, store_returns_quantity#35, catalog_sales_quantity#36] + +(42) ColumnarToRow [codegen id : 9] +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#34, store_returns_quantity#35, catalog_sales_quantity#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#40, d_moy#41] +Output [3]: [d_date_sk#19, d_year#37, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] -Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 9)) AND (d_year#40 = 1999)) AND isnotnull(d_date_sk#19)) +(44) CometFilter +Input [3]: [d_date_sk#19, d_year#37, d_moy#38] +Condition : ((((isnotnull(d_moy#38) AND isnotnull(d_year#37)) AND (d_moy#38 = 9)) AND (d_year#37 = 1999)) AND isnotnull(d_date_sk#19)) -(43) CometProject -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] +(45) CometProject +Input [3]: [d_date_sk#19, d_year#37, d_moy#38] Arguments: [d_date_sk#19], [d_date_sk#19] -(44) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(45) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#42, d_moy#43] +Output [3]: [d_date_sk#20, d_year#39, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] -Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 9)) AND (d_moy#43 <= 12)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#20)) +(49) CometFilter +Input [3]: [d_date_sk#20, d_year#39, d_moy#40] +Condition : (((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 >= 9)) AND (d_moy#40 <= 12)) AND (d_year#39 = 1999)) AND isnotnull(d_date_sk#20)) -(48) CometProject -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] +(50) CometProject +Input [3]: [d_date_sk#20, d_year#39, d_moy#40] Arguments: [d_date_sk#20], [d_date_sk#20] -(49) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#20] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_year#44] +Output [2]: [d_date_sk#21, d_year#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter -Input [2]: [d_date_sk#21, d_year#44] -Condition : (d_year#44 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) +(54) CometFilter +Input [2]: [d_date_sk#21, d_year#41] +Condition : (d_year#41 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) -(53) CometProject -Input [2]: [d_date_sk#21, d_year#44] +(55) CometProject +Input [2]: [d_date_sk#21, d_year#41] Arguments: [d_date_sk#21], [d_date_sk#21] -(54) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#21] -(55) BroadcastExchange +(57) BroadcastExchange Input [1]: [d_date_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt index 68a127d357..753b11f806 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt @@ -1,83 +1,85 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] + CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] + Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #6 InputAdapter - ReusedExchange [d_date_sk] #6 + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) + BroadcastExchange #8 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt index 22b8681f2a..2baeadfd8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt @@ -1,25 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +* ColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometHashAggregate (21) + +- CometColumnarExchange (20) + +- RowToColumnar (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) (unknown) Scan parquet spark_catalog.default.date_dim @@ -108,18 +110,22 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -(19) Exchange +(19) RowToColumnar Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) HashAggregate [codegen id : 4] +(20) CometColumnarExchange +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(21) CometHashAggregate Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum_agg#16] -(21) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#14, brand#15, sum_agg#16] -Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, sum_agg#16] +(22) CometTakeOrderedAndProject +Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), 100, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] + +(23) ColumnarToRow [codegen id : 4] +Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt index 3946c0cd8f..a627b6b25e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt @@ -1,31 +1,33 @@ -TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] - WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum] - InputAdapter - Exchange [d_year,i_brand,i_brand_id] #1 - WholeStageCodegen (3) - HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,sum_agg,brand_id,brand] + CometHashAggregate [d_year,i_brand,i_brand_id,sum] + CometColumnarExchange [d_year,i_brand,i_brand_id] #1 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year] + CometFilter [d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index 5da800fe40..13d06f87c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt @@ -1,53 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (33) - : : +- * Filter (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * HashAggregate (28) - : : +- Exchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.web_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : +- BroadcastExchange (39) - : +- * ColumnarToRow (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.customer (36) - +- BroadcastExchange (46) - +- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.customer_address (42) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometColumnarExchange (15) + : : : +- RowToColumnar (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan parquet spark_catalog.default.customer_address (7) + : : +- BroadcastExchange (37) + : : +- * ColumnarToRow (36) + : : +- CometFilter (35) + : : +- CometHashAggregate (34) + : : +- CometColumnarExchange (33) + : : +- CometHashAggregate (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- RowToColumnar (29) + : : +- * HashAggregate (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (24) + : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : :- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.web_returns (19) + : : : +- ReusedExchange (22) + : : +- ReusedExchange (25) + : +- BroadcastExchange (43) + : +- * ColumnarToRow (42) + : +- CometFilter (41) + : +- CometScan parquet spark_catalog.default.customer (40) + +- BroadcastExchange (50) + +- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.customer_address (46) (unknown) Scan parquet spark_catalog.default.web_returns @@ -65,7 +69,7 @@ Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_custom (3) ColumnarToRow [codegen id : 3] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -(4) ReusedExchange [Reuses operator id: 54] +(4) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -113,212 +117,216 @@ Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum#9] Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] -(14) Exchange +(14) RowToColumnar Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 11] +(15) CometColumnarExchange +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] Keys [2]: [wr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#11] -Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] -(16) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) +(17) CometFilter +Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +Condition : isnotnull(ctr_total_return#13) + +(18) ColumnarToRow [codegen id : 10] +Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] (unknown) Scan parquet spark_catalog.default.web_returns Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#14)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(18) CometFilter +(20) CometFilter Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Condition : isnotnull(wr_returning_addr_sk#2) -(19) ColumnarToRow [codegen id : 6] +(21) ColumnarToRow [codegen id : 6] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -(20) ReusedExchange [Reuses operator id: 54] +(22) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#6] -(21) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [wr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] -(23) ReusedExchange [Reuses operator id: 10] +(25) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#7, ca_state#8] -(24) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [wr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(27) Project [codegen id : 6] Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#8] -(26) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 6] Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] Keys [2]: [wr_returning_customer_sk#1, ca_state#8] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#16] -Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#16] -(27) Exchange -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(29) RowToColumnar +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#16] -(28) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] +(30) CometColumnarExchange +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#16] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(31) CometHashAggregate +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#16] Keys [2]: [wr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#11] -Results [2]: [ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] - -(29) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#13, ctr_total_return#14] -Keys [1]: [ctr_state#13] -Functions [1]: [partial_avg(ctr_total_return#14)] -Aggregate Attributes [2]: [sum#18, count#19] -Results [3]: [ctr_state#13, sum#20, count#21] - -(30) Exchange -Input [3]: [ctr_state#13, sum#20, count#21] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#13, sum#20, count#21] -Keys [1]: [ctr_state#13] -Functions [1]: [avg(ctr_total_return#14)] -Aggregate Attributes [1]: [avg(ctr_total_return#14)#22] -Results [2]: [(avg(ctr_total_return#14)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#13 AS ctr_state#13#24] - -(32) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) - -(33) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] + +(32) CometHashAggregate +Input [2]: [ctr_state#12, ctr_total_return#13] +Keys [1]: [ctr_state#12] +Functions [1]: [partial_avg(ctr_total_return#13)] + +(33) CometColumnarExchange +Input [3]: [ctr_state#12, sum#17, count#18] +Arguments: hashpartitioning(ctr_state#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometHashAggregate +Input [3]: [ctr_state#12, sum#17, count#18] +Keys [1]: [ctr_state#12] +Functions [1]: [avg(ctr_total_return#13)] + +(35) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#19) + +(36) ColumnarToRow [codegen id : 7] +Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] + +(37) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#13] -Right keys [1]: [ctr_state#13#24] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ctr_state#12] +Right keys [1]: [ctr_state#12#20] Join type: Inner -Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) +Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#19) -(35) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] +(39) Project [codegen id : 10] +Output [2]: [ctr_customer_sk#11, ctr_total_return#13] +Input [5]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] (unknown) Scan parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] +Output [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(37) CometFilter -Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) +(41) CometFilter +Input [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#23)) -(38) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] +(42) ColumnarToRow [codegen id : 8] +Input [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] -(39) BroadcastExchange -Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] +(43) BroadcastExchange +Input [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#12] -Right keys [1]: [c_customer_sk#25] +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ctr_customer_sk#11] +Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(41) Project [codegen id : 11] -Output [14]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] -Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] +(45) Project [codegen id : 10] +Output [14]: [ctr_total_return#13, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] +Input [16]: [ctr_customer_sk#11, ctr_total_return#13, c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] (unknown) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#39, ca_state#40] +Output [2]: [ca_address_sk#35, ca_state#36] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(43) CometFilter -Input [2]: [ca_address_sk#39, ca_state#40] -Condition : ((isnotnull(ca_state#40) AND (ca_state#40 = GA)) AND isnotnull(ca_address_sk#39)) +(47) CometFilter +Input [2]: [ca_address_sk#35, ca_state#36] +Condition : ((isnotnull(ca_state#36) AND (ca_state#36 = GA)) AND isnotnull(ca_address_sk#35)) -(44) CometProject -Input [2]: [ca_address_sk#39, ca_state#40] -Arguments: [ca_address_sk#39], [ca_address_sk#39] +(48) CometProject +Input [2]: [ca_address_sk#35, ca_state#36] +Arguments: [ca_address_sk#35], [ca_address_sk#35] -(45) ColumnarToRow [codegen id : 10] -Input [1]: [ca_address_sk#39] +(49) ColumnarToRow [codegen id : 9] +Input [1]: [ca_address_sk#35] -(46) BroadcastExchange -Input [1]: [ca_address_sk#39] +(50) BroadcastExchange +Input [1]: [ca_address_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#27] -Right keys [1]: [ca_address_sk#39] +(51) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_current_addr_sk#23] +Right keys [1]: [ca_address_sk#35] Join type: Inner Join condition: None -(48) Project [codegen id : 11] -Output [13]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] -Input [15]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ca_address_sk#39] +(52) Project [codegen id : 10] +Output [13]: [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34, ctr_total_return#13] +Input [15]: [ctr_total_return#13, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34, ca_address_sk#35] -(49) TakeOrderedAndProject -Input [13]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] -Arguments: 100, [c_customer_id#26 ASC NULLS FIRST, c_salutation#28 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, c_last_name#30 ASC NULLS FIRST, c_preferred_cust_flag#31 ASC NULLS FIRST, c_birth_day#32 ASC NULLS FIRST, c_birth_month#33 ASC NULLS FIRST, c_birth_year#34 ASC NULLS FIRST, c_birth_country#35 ASC NULLS FIRST, c_login#36 ASC NULLS FIRST, c_email_address#37 ASC NULLS FIRST, c_last_review_date#38 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] +(53) TakeOrderedAndProject +Input [13]: [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34, ctr_total_return#13] +Arguments: 100, [c_customer_id#22 ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_first_name#25 ASC NULLS FIRST, c_last_name#26 ASC NULLS FIRST, c_preferred_cust_flag#27 ASC NULLS FIRST, c_birth_day#28 ASC NULLS FIRST, c_birth_month#29 ASC NULLS FIRST, c_birth_year#30 ASC NULLS FIRST, c_birth_country#31 ASC NULLS FIRST, c_login#32 ASC NULLS FIRST, c_email_address#33 ASC NULLS FIRST, c_last_review_date#34 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34, ctr_total_return#13] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +BroadcastExchange (58) ++- * ColumnarToRow (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#41] +Output [2]: [d_date_sk#6, d_year#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter -Input [2]: [d_date_sk#6, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#6)) +(55) CometFilter +Input [2]: [d_date_sk#6, d_year#37] +Condition : ((isnotnull(d_year#37) AND (d_year#37 = 2002)) AND isnotnull(d_date_sk#6)) -(52) CometProject -Input [2]: [d_date_sk#6, d_year#41] +(56) CometProject +Input [2]: [d_date_sk#6, d_year#37] Arguments: [d_date_sk#6], [d_date_sk#6] -(53) ColumnarToRow [codegen id : 1] +(57) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(54) BroadcastExchange +(58) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 17 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index c70f33da33..a883ad9d96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -1,54 +1,56 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (11) + WholeStageCodegen (10) Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk,ctr_total_return] BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] - CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow + ColumnarToRow + InputAdapter + CometFilter [ctr_total_return] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum] + CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 - WholeStageCodegen (8) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] - InputAdapter - Exchange [ctr_state] #5 - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [wr_returning_customer_sk,ca_state] #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [(avg(ctr_total_return) * 1.2)] + CometHashAggregate [ctr_state,sum,count] + CometColumnarExchange [ctr_state] #5 + CometHashAggregate [ctr_state,ctr_total_return] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum] + CometColumnarExchange [wr_returning_customer_sk,ca_state] #6 + RowToColumnar WholeStageCodegen (6) HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] Project [wr_returning_customer_sk,wr_return_amt,ca_state] @@ -66,14 +68,14 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 - WholeStageCodegen (9) + WholeStageCodegen (8) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #8 - WholeStageCodegen (10) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometProject [ca_address_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index 7b1f7bb5db..ccfa5eb025 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt @@ -1,94 +1,108 @@ == Physical Plan == -* Sort (90) -+- Exchange (89) - +- * Project (88) - +- * BroadcastHashJoin Inner BuildRight (87) - :- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * BroadcastHashJoin Inner BuildRight (29) - : : : : :- * HashAggregate (15) - : : : : : +- Exchange (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (6) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- ReusedExchange (4) - : : : : : +- BroadcastExchange (10) - : : : : : +- * ColumnarToRow (9) - : : : : : +- CometFilter (8) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : : : +- BroadcastExchange (28) - : : : : +- * HashAggregate (27) - : : : : +- Exchange (26) - : : : : +- * HashAggregate (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : :- * Project (21) - : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : :- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (16) - : : : : : +- ReusedExchange (19) - : : : : +- ReusedExchange (22) - : : : +- BroadcastExchange (42) - : : : +- * HashAggregate (41) - : : : +- Exchange (40) - : : : +- * HashAggregate (39) - : : : +- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * ColumnarToRow (32) - : : : : : +- CometFilter (31) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (33) - : : : +- ReusedExchange (36) - : : +- BroadcastExchange (57) - : : +- * HashAggregate (56) - : : +- Exchange (55) - : : +- * HashAggregate (54) - : : +- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * ColumnarToRow (47) - : : : : +- CometFilter (46) - : : : : +- CometScan parquet spark_catalog.default.web_sales (45) - : : : +- ReusedExchange (48) - : : +- ReusedExchange (51) - : +- BroadcastExchange (71) - : +- * HashAggregate (70) - : +- Exchange (69) - : +- * HashAggregate (68) - : +- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * ColumnarToRow (61) - : : : +- CometFilter (60) - : : : +- CometScan parquet spark_catalog.default.web_sales (59) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- BroadcastExchange (86) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * BroadcastHashJoin Inner BuildRight (81) - :- * Project (79) - : +- * BroadcastHashJoin Inner BuildRight (78) - : :- * ColumnarToRow (76) - : : +- CometFilter (75) - : : +- CometScan parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (77) - +- ReusedExchange (80) +* ColumnarToRow (104) ++- CometSort (103) + +- CometColumnarExchange (102) + +- RowToColumnar (101) + +- * Project (100) + +- * BroadcastHashJoin Inner BuildRight (99) + :- * Project (83) + : +- * BroadcastHashJoin Inner BuildRight (82) + : :- * BroadcastHashJoin Inner BuildRight (66) + : : :- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * ColumnarToRow (17) + : : : : : +- CometHashAggregate (16) + : : : : : +- CometColumnarExchange (15) + : : : : : +- RowToColumnar (14) + : : : : : +- * HashAggregate (13) + : : : : : +- * Project (12) + : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : :- * Project (6) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- ReusedExchange (4) + : : : : : +- BroadcastExchange (10) + : : : : : +- * ColumnarToRow (9) + : : : : : +- CometFilter (8) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (7) + : : : : +- BroadcastExchange (32) + : : : : +- * ColumnarToRow (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometColumnarExchange (29) + : : : : +- RowToColumnar (28) + : : : : +- * HashAggregate (27) + : : : : +- * Project (26) + : : : : +- * BroadcastHashJoin Inner BuildRight (25) + : : : : :- * Project (23) + : : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : : :- * ColumnarToRow (20) + : : : : : : +- CometFilter (19) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (18) + : : : : : +- ReusedExchange (21) + : : : : +- ReusedExchange (24) + : : : +- BroadcastExchange (48) + : : : +- * ColumnarToRow (47) + : : : +- CometHashAggregate (46) + : : : +- CometColumnarExchange (45) + : : : +- RowToColumnar (44) + : : : +- * HashAggregate (43) + : : : +- * Project (42) + : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : :- * Project (39) + : : : : +- * BroadcastHashJoin Inner BuildRight (38) + : : : : :- * ColumnarToRow (36) + : : : : : +- CometFilter (35) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (34) + : : : : +- ReusedExchange (37) + : : : +- ReusedExchange (40) + : : +- BroadcastExchange (65) + : : +- * ColumnarToRow (64) + : : +- CometHashAggregate (63) + : : +- CometColumnarExchange (62) + : : +- RowToColumnar (61) + : : +- * HashAggregate (60) + : : +- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * Project (56) + : : : +- * BroadcastHashJoin Inner BuildRight (55) + : : : :- * ColumnarToRow (53) + : : : : +- CometFilter (52) + : : : : +- CometScan parquet spark_catalog.default.web_sales (51) + : : : +- ReusedExchange (54) + : : +- ReusedExchange (57) + : +- BroadcastExchange (81) + : +- * ColumnarToRow (80) + : +- CometHashAggregate (79) + : +- CometColumnarExchange (78) + : +- RowToColumnar (77) + : +- * HashAggregate (76) + : +- * Project (75) + : +- * BroadcastHashJoin Inner BuildRight (74) + : :- * Project (72) + : : +- * BroadcastHashJoin Inner BuildRight (71) + : : :- * ColumnarToRow (69) + : : : +- CometFilter (68) + : : : +- CometScan parquet spark_catalog.default.web_sales (67) + : : +- ReusedExchange (70) + : +- ReusedExchange (73) + +- BroadcastExchange (98) + +- * ColumnarToRow (97) + +- CometHashAggregate (96) + +- CometColumnarExchange (95) + +- RowToColumnar (94) + +- * HashAggregate (93) + +- * Project (92) + +- * BroadcastHashJoin Inner BuildRight (91) + :- * Project (89) + : +- * BroadcastHashJoin Inner BuildRight (88) + : :- * ColumnarToRow (86) + : : +- CometFilter (85) + : : +- CometScan parquet spark_catalog.default.web_sales (84) + : +- ReusedExchange (87) + +- ReusedExchange (90) (unknown) Scan parquet spark_catalog.default.store_sales @@ -106,7 +120,7 @@ Condition : isnotnull(ss_addr_sk#1) (3) ColumnarToRow [codegen id : 3] Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 94] +(4) ReusedExchange [Reuses operator id: 108] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] (5) BroadcastHashJoin [codegen id : 3] @@ -154,389 +168,419 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#10] Results [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -(14) Exchange +(14) RowToColumnar Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 24] +(15) CometColumnarExchange +Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] +Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] -Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS store_sales#13] + +(17) ColumnarToRow [codegen id : 24] +Input [3]: [ca_county#9, d_year#6, store_sales#12] (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] +Output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(17) CometFilter -Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -Condition : isnotnull(ss_addr_sk#14) +(19) CometFilter +Input [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] +Condition : isnotnull(ss_addr_sk#13) -(18) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] +(20) ColumnarToRow [codegen id : 6] +Input [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] -(19) ReusedExchange [Reuses operator id: 98] -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +(21) ReusedExchange [Reuses operator id: 112] +Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#16] -Right keys [1]: [d_date_sk#18] +(22) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#15] +Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(21) Project [codegen id : 6] -Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] -Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] +(23) Project [codegen id : 6] +Output [4]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19] +Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_year#18, d_qoy#19] -(22) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#21, ca_county#22] +(24) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#20, ca_county#21] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#14] -Right keys [1]: [ca_address_sk#21] +(25) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_addr_sk#13] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] - -(25) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -Keys [3]: [ca_county#22, d_qoy#20, d_year#19] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] -Aggregate Attributes [1]: [sum#23] -Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] - -(26) Exchange -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 7] -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Keys [3]: [ca_county#22, d_qoy#20, d_year#19] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#15))#12] -Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#12,17,2) AS store_sales#25] - -(28) BroadcastExchange -Input [2]: [ca_county#22, store_sales#25] +(26) Project [codegen id : 6] +Output [4]: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] +Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_address_sk#20, ca_county#21] + +(27) HashAggregate [codegen id : 6] +Input [4]: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] +Keys [3]: [ca_county#21, d_qoy#19, d_year#18] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#14))] +Aggregate Attributes [1]: [sum#22] +Results [4]: [ca_county#21, d_qoy#19, d_year#18, sum#23] + +(28) RowToColumnar +Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#23] + +(29) CometColumnarExchange +Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#23] +Arguments: hashpartitioning(ca_county#21, d_qoy#19, d_year#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(30) CometHashAggregate +Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#23] +Keys [3]: [ca_county#21, d_qoy#19, d_year#18] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#14))] + +(31) ColumnarToRow [codegen id : 7] +Input [2]: [ca_county#21, store_sales#24] + +(32) BroadcastExchange +Input [2]: [ca_county#21, store_sales#24] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 24] +(33) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#22] +Right keys [1]: [ca_county#21] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Output [3]: [ss_addr_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(31) CometFilter -Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : isnotnull(ss_addr_sk#26) +(35) CometFilter +Input [3]: [ss_addr_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27] +Condition : isnotnull(ss_addr_sk#25) -(32) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +(36) ColumnarToRow [codegen id : 10] +Input [3]: [ss_addr_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27] -(33) ReusedExchange [Reuses operator id: 102] -Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] +(37) ReusedExchange [Reuses operator id: 116] +Output [3]: [d_date_sk#29, d_year#30, d_qoy#31] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#30] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#27] +Right keys [1]: [d_date_sk#29] Join type: Inner Join condition: None -(35) Project [codegen id : 10] -Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] -Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] +(39) Project [codegen id : 10] +Output [4]: [ss_addr_sk#25, ss_ext_sales_price#26, d_year#30, d_qoy#31] +Input [6]: [ss_addr_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27, d_date_sk#29, d_year#30, d_qoy#31] -(36) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#33, ca_county#34] +(40) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#32, ca_county#33] -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#26] -Right keys [1]: [ca_address_sk#33] +(41) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_addr_sk#25] +Right keys [1]: [ca_address_sk#32] Join type: Inner Join condition: None -(38) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] - -(39) HashAggregate [codegen id : 10] -Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -Keys [3]: [ca_county#34, d_qoy#32, d_year#31] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#35] -Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] - -(40) Exchange -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(41) HashAggregate [codegen id : 11] -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Keys [3]: [ca_county#34, d_qoy#32, d_year#31] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#12] -Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#12,17,2) AS store_sales#37] - -(42) BroadcastExchange -Input [2]: [ca_county#34, store_sales#37] +(42) Project [codegen id : 10] +Output [4]: [ss_ext_sales_price#26, d_year#30, d_qoy#31, ca_county#33] +Input [6]: [ss_addr_sk#25, ss_ext_sales_price#26, d_year#30, d_qoy#31, ca_address_sk#32, ca_county#33] + +(43) HashAggregate [codegen id : 10] +Input [4]: [ss_ext_sales_price#26, d_year#30, d_qoy#31, ca_county#33] +Keys [3]: [ca_county#33, d_qoy#31, d_year#30] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#26))] +Aggregate Attributes [1]: [sum#34] +Results [4]: [ca_county#33, d_qoy#31, d_year#30, sum#35] + +(44) RowToColumnar +Input [4]: [ca_county#33, d_qoy#31, d_year#30, sum#35] + +(45) CometColumnarExchange +Input [4]: [ca_county#33, d_qoy#31, d_year#30, sum#35] +Arguments: hashpartitioning(ca_county#33, d_qoy#31, d_year#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(46) CometHashAggregate +Input [4]: [ca_county#33, d_qoy#31, d_year#30, sum#35] +Keys [3]: [ca_county#33, d_qoy#31, d_year#30] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#26))] + +(47) ColumnarToRow [codegen id : 11] +Input [2]: [ca_county#33, store_sales#36] + +(48) BroadcastExchange +Input [2]: [ca_county#33, store_sales#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#22] -Right keys [1]: [ca_county#34] +(49) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ca_county#21] +Right keys [1]: [ca_county#33] Join type: Inner Join condition: None -(44) Project [codegen id : 24] -Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] -Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] +(50) Project [codegen id : 24] +Output [5]: [ca_county#9, d_year#6, store_sales#12, store_sales#24, store_sales#36] +Input [7]: [ca_county#9, d_year#6, store_sales#12, ca_county#21, store_sales#24, ca_county#33, store_sales#36] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Output [3]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, ws_sold_date_sk#39] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(ws_sold_date_sk#39), dynamicpruningexpression(ws_sold_date_sk#39 IN dynamicpruning#40)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(46) CometFilter -Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_addr_sk#38) +(52) CometFilter +Input [3]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, ws_sold_date_sk#39] +Condition : isnotnull(ws_bill_addr_sk#37) -(47) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(53) ColumnarToRow [codegen id : 14] +Input [3]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, ws_sold_date_sk#39] -(48) ReusedExchange [Reuses operator id: 94] -Output [3]: [d_date_sk#42, d_year#43, d_qoy#44] +(54) ReusedExchange [Reuses operator id: 108] +Output [3]: [d_date_sk#41, d_year#42, d_qoy#43] -(49) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#42] +(55) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#39] +Right keys [1]: [d_date_sk#41] Join type: Inner Join condition: None -(50) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44] -Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#42, d_year#43, d_qoy#44] +(56) Project [codegen id : 14] +Output [4]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, d_year#42, d_qoy#43] +Input [6]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, ws_sold_date_sk#39, d_date_sk#41, d_year#42, d_qoy#43] -(51) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#45, ca_county#46] +(57) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#44, ca_county#45] -(52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#45] +(58) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_bill_addr_sk#37] +Right keys [1]: [ca_address_sk#44] Join type: Inner Join condition: None -(53) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] -Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_address_sk#45, ca_county#46] - -(54) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] -Keys [3]: [ca_county#46, d_qoy#44, d_year#43] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#47] -Results [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] - -(55) Exchange -Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] -Arguments: hashpartitioning(ca_county#46, d_qoy#44, d_year#43, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(56) HashAggregate [codegen id : 15] -Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] -Keys [3]: [ca_county#46, d_qoy#44, d_year#43] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#49] -Results [2]: [ca_county#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#49,17,2) AS web_sales#50] - -(57) BroadcastExchange -Input [2]: [ca_county#46, web_sales#50] +(59) Project [codegen id : 14] +Output [4]: [ws_ext_sales_price#38, d_year#42, d_qoy#43, ca_county#45] +Input [6]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, d_year#42, d_qoy#43, ca_address_sk#44, ca_county#45] + +(60) HashAggregate [codegen id : 14] +Input [4]: [ws_ext_sales_price#38, d_year#42, d_qoy#43, ca_county#45] +Keys [3]: [ca_county#45, d_qoy#43, d_year#42] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#38))] +Aggregate Attributes [1]: [sum#46] +Results [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] + +(61) RowToColumnar +Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] + +(62) CometColumnarExchange +Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] +Arguments: hashpartitioning(ca_county#45, d_qoy#43, d_year#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(63) CometHashAggregate +Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] +Keys [3]: [ca_county#45, d_qoy#43, d_year#42] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#38))] + +(64) ColumnarToRow [codegen id : 15] +Input [2]: [ca_county#45, web_sales#48] + +(65) BroadcastExchange +Input [2]: [ca_county#45, web_sales#48] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(58) BroadcastHashJoin [codegen id : 24] +(66) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#46] +Right keys [1]: [ca_county#45] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] +Output [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#52)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(60) CometFilter -Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_bill_addr_sk#51) +(68) CometFilter +Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_addr_sk#49) -(61) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] +(69) ColumnarToRow [codegen id : 18] +Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -(62) ReusedExchange [Reuses operator id: 98] -Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] +(70) ReusedExchange [Reuses operator id: 112] +Output [3]: [d_date_sk#53, d_year#54, d_qoy#55] -(63) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#53] -Right keys [1]: [d_date_sk#55] +(71) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#53] Join type: Inner Join condition: None -(64) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57] -Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56, d_qoy#57] +(72) Project [codegen id : 18] +Output [4]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#54, d_qoy#55] +Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51, d_date_sk#53, d_year#54, d_qoy#55] -(65) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#58, ca_county#59] +(73) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#56, ca_county#57] -(66) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#51] -Right keys [1]: [ca_address_sk#58] +(74) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_bill_addr_sk#49] +Right keys [1]: [ca_address_sk#56] Join type: Inner Join condition: None -(67) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] -Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_address_sk#58, ca_county#59] - -(68) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] -Keys [3]: [ca_county#59, d_qoy#57, d_year#56] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#52))] -Aggregate Attributes [1]: [sum#60] -Results [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] - -(69) Exchange -Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] -Arguments: hashpartitioning(ca_county#59, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(70) HashAggregate [codegen id : 19] -Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] -Keys [3]: [ca_county#59, d_qoy#57, d_year#56] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#52))#49] -Results [2]: [ca_county#59, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#52))#49,17,2) AS web_sales#62] - -(71) BroadcastExchange -Input [2]: [ca_county#59, web_sales#62] +(75) Project [codegen id : 18] +Output [4]: [ws_ext_sales_price#50, d_year#54, d_qoy#55, ca_county#57] +Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#54, d_qoy#55, ca_address_sk#56, ca_county#57] + +(76) HashAggregate [codegen id : 18] +Input [4]: [ws_ext_sales_price#50, d_year#54, d_qoy#55, ca_county#57] +Keys [3]: [ca_county#57, d_qoy#55, d_year#54] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] +Aggregate Attributes [1]: [sum#58] +Results [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] + +(77) RowToColumnar +Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] + +(78) CometColumnarExchange +Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] +Arguments: hashpartitioning(ca_county#57, d_qoy#55, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(79) CometHashAggregate +Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] +Keys [3]: [ca_county#57, d_qoy#55, d_year#54] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#50))] + +(80) ColumnarToRow [codegen id : 19] +Input [2]: [ca_county#57, web_sales#60] + +(81) BroadcastExchange +Input [2]: [ca_county#57, web_sales#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] -(72) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#46] -Right keys [1]: [ca_county#59] +(82) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ca_county#45] +Right keys [1]: [ca_county#57] Join type: Inner -Join condition: (CASE WHEN (web_sales#50 > 0.00) THEN (web_sales#62 / web_sales#50) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) +Join condition: (CASE WHEN (web_sales#48 > 0.00) THEN (web_sales#60 / web_sales#48) END > CASE WHEN (store_sales#12 > 0.00) THEN (store_sales#24 / store_sales#12) END) -(73) Project [codegen id : 24] -Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62] -Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, ca_county#59, web_sales#62] +(83) Project [codegen id : 24] +Output [8]: [ca_county#9, d_year#6, store_sales#12, store_sales#24, store_sales#36, ca_county#45, web_sales#48, web_sales#60] +Input [9]: [ca_county#9, d_year#6, store_sales#12, store_sales#24, store_sales#36, ca_county#45, web_sales#48, ca_county#57, web_sales#60] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Output [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#66)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(75) CometFilter -Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_bill_addr_sk#63) +(85) CometFilter +Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_addr_sk#61) -(76) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +(86) ColumnarToRow [codegen id : 22] +Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -(77) ReusedExchange [Reuses operator id: 102] -Output [3]: [d_date_sk#67, d_year#68, d_qoy#69] +(87) ReusedExchange [Reuses operator id: 116] +Output [3]: [d_date_sk#65, d_year#66, d_qoy#67] -(78) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#67] +(88) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(79) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69] -Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68, d_qoy#69] +(89) Project [codegen id : 22] +Output [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#66, d_qoy#67] +Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66, d_qoy#67] -(80) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#70, ca_county#71] +(90) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#68, ca_county#69] -(81) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#63] -Right keys [1]: [ca_address_sk#70] +(91) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_bill_addr_sk#61] +Right keys [1]: [ca_address_sk#68] Join type: Inner Join condition: None -(82) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] -Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_address_sk#70, ca_county#71] - -(83) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] -Keys [3]: [ca_county#71, d_qoy#69, d_year#68] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] -Aggregate Attributes [1]: [sum#72] -Results [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] - -(84) Exchange -Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] -Arguments: hashpartitioning(ca_county#71, d_qoy#69, d_year#68, 5), ENSURE_REQUIREMENTS, [plan_id=11] - -(85) HashAggregate [codegen id : 23] -Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] -Keys [3]: [ca_county#71, d_qoy#69, d_year#68] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#64))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#64))#49] -Results [2]: [ca_county#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#49,17,2) AS web_sales#74] - -(86) BroadcastExchange -Input [2]: [ca_county#71, web_sales#74] +(92) Project [codegen id : 22] +Output [4]: [ws_ext_sales_price#62, d_year#66, d_qoy#67, ca_county#69] +Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#66, d_qoy#67, ca_address_sk#68, ca_county#69] + +(93) HashAggregate [codegen id : 22] +Input [4]: [ws_ext_sales_price#62, d_year#66, d_qoy#67, ca_county#69] +Keys [3]: [ca_county#69, d_qoy#67, d_year#66] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#62))] +Aggregate Attributes [1]: [sum#70] +Results [4]: [ca_county#69, d_qoy#67, d_year#66, sum#71] + +(94) RowToColumnar +Input [4]: [ca_county#69, d_qoy#67, d_year#66, sum#71] + +(95) CometColumnarExchange +Input [4]: [ca_county#69, d_qoy#67, d_year#66, sum#71] +Arguments: hashpartitioning(ca_county#69, d_qoy#67, d_year#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(96) CometHashAggregate +Input [4]: [ca_county#69, d_qoy#67, d_year#66, sum#71] +Keys [3]: [ca_county#69, d_qoy#67, d_year#66] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#62))] + +(97) ColumnarToRow [codegen id : 23] +Input [2]: [ca_county#69, web_sales#72] + +(98) BroadcastExchange +Input [2]: [ca_county#69, web_sales#72] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] -(87) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#46] -Right keys [1]: [ca_county#71] +(99) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ca_county#45] +Right keys [1]: [ca_county#69] Join type: Inner -Join condition: (CASE WHEN (web_sales#62 > 0.00) THEN (web_sales#74 / web_sales#62) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) +Join condition: (CASE WHEN (web_sales#60 > 0.00) THEN (web_sales#72 / web_sales#60) END > CASE WHEN (store_sales#24 > 0.00) THEN (store_sales#36 / store_sales#24) END) + +(100) Project [codegen id : 24] +Output [6]: [ca_county#9, d_year#6, (web_sales#60 / web_sales#48) AS web_q1_q2_increase#73, (store_sales#24 / store_sales#12) AS store_q1_q2_increase#74, (web_sales#72 / web_sales#60) AS web_q2_q3_increase#75, (store_sales#36 / store_sales#24) AS store_q2_q3_increase#76] +Input [10]: [ca_county#9, d_year#6, store_sales#12, store_sales#24, store_sales#36, ca_county#45, web_sales#48, web_sales#60, ca_county#69, web_sales#72] -(88) Project [codegen id : 24] -Output [6]: [ca_county#9, d_year#6, (web_sales#62 / web_sales#50) AS web_q1_q2_increase#75, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#76, (web_sales#74 / web_sales#62) AS web_q2_q3_increase#77, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#78] -Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62, ca_county#71, web_sales#74] +(101) RowToColumnar +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#73, store_q1_q2_increase#74, web_q2_q3_increase#75, store_q2_q3_increase#76] -(89) Exchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=13] +(102) CometColumnarExchange +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#73, store_q1_q2_increase#74, web_q2_q3_increase#75, store_q2_q3_increase#76] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(90) Sort [codegen id : 25] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] -Arguments: [ca_county#9 ASC NULLS FIRST], true, 0 +(103) CometSort +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#73, store_q1_q2_increase#74, web_q2_q3_increase#75, store_q2_q3_increase#76] +Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#73, store_q1_q2_increase#74, web_q2_q3_increase#75, store_q2_q3_increase#76], [ca_county#9 ASC NULLS FIRST] + +(104) ColumnarToRow [codegen id : 25] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#73, store_q1_q2_increase#74, web_q2_q3_increase#75, store_q2_q3_increase#76] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (94) -+- * ColumnarToRow (93) - +- CometFilter (92) - +- CometScan parquet spark_catalog.default.date_dim (91) +BroadcastExchange (108) ++- * ColumnarToRow (107) + +- CometFilter (106) + +- CometScan parquet spark_catalog.default.date_dim (105) (unknown) Scan parquet spark_catalog.default.date_dim @@ -546,71 +590,71 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(92) CometFilter +(106) CometFilter Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) ColumnarToRow [codegen id : 1] +(107) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(94) BroadcastExchange +(108) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (98) -+- * ColumnarToRow (97) - +- CometFilter (96) - +- CometScan parquet spark_catalog.default.date_dim (95) +Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#16 +BroadcastExchange (112) ++- * ColumnarToRow (111) + +- CometFilter (110) + +- CometScan parquet spark_catalog.default.date_dim (109) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(96) CometFilter -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) +(110) CometFilter +Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] +Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) -(97) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +(111) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -(98) BroadcastExchange -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +(112) BroadcastExchange +Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 -BroadcastExchange (102) -+- * ColumnarToRow (101) - +- CometFilter (100) - +- CometScan parquet spark_catalog.default.date_dim (99) +Subquery:3 Hosting operator id = 34 Hosting Expression = ss_sold_date_sk#27 IN dynamicpruning#28 +BroadcastExchange (116) ++- * ColumnarToRow (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] +Output [3]: [d_date_sk#29, d_year#30, d_qoy#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(100) CometFilter -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) +(114) CometFilter +Input [3]: [d_date_sk#29, d_year#30, d_qoy#31] +Condition : ((((isnotnull(d_qoy#31) AND isnotnull(d_year#30)) AND (d_qoy#31 = 3)) AND (d_year#30 = 2000)) AND isnotnull(d_date_sk#29)) -(101) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] +(115) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#29, d_year#30, d_qoy#31] -(102) BroadcastExchange -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] +(116) BroadcastExchange +Input [3]: [d_date_sk#29, d_year#30, d_qoy#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:4 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 59 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#17 +Subquery:5 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#16 -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#29 +Subquery:6 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#28 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index f4bf6a89d0..925a4a7f30 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -1,159 +1,173 @@ WholeStageCodegen (25) - Sort [ca_county] + ColumnarToRow InputAdapter - Exchange [ca_county] #1 - WholeStageCodegen (24) - Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] + CometSort [ca_county] + CometColumnarExchange [ca_county] #1 + RowToColumnar + WholeStageCodegen (24) + Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] + Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) + Project [ca_county,d_year,store_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county] + BroadcastHashJoin [ca_county,ca_county] + ColumnarToRow + InputAdapter + CometHashAggregate [ca_county,d_qoy,d_year,sum] + CometColumnarExchange [ca_county,d_qoy,d_year] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_county,d_qoy,d_year,sum] + CometColumnarExchange [ca_county,d_qoy,d_year] #6 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow + CometFilter [ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #7 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_county,d_qoy,d_year,sum] + CometColumnarExchange [ca_county,d_qoy,d_year] #9 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #10 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #10 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_county,d_qoy,d_year,sum] + CometColumnarExchange [ca_county,d_qoy,d_year] #12 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 InputAdapter - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_county,d_qoy,d_year,sum] + CometColumnarExchange [ca_county,d_qoy,d_year] #14 + RowToColumnar + WholeStageCodegen (18) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometFilter [ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk,d_year,d_qoy] #7 InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + BroadcastExchange #15 + WholeStageCodegen (23) + ColumnarToRow InputAdapter - Exchange [ca_county,d_qoy,d_year] #14 - WholeStageCodegen (18) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + CometHashAggregate [ca_county,d_qoy,d_year,sum] + CometColumnarExchange [ca_county,d_qoy,d_year] #16 + RowToColumnar + WholeStageCodegen (22) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #10 InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #16 - WholeStageCodegen (22) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [ca_address_sk,ca_county] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index e3c659e278..c9ff159239 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -1,33 +1,37 @@ == Physical Plan == -* HashAggregate (29) -+- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * HashAggregate (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * ColumnarToRow (13) - : : +- CometFilter (12) - : : +- CometScan parquet spark_catalog.default.catalog_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (24) +* ColumnarToRow (33) ++- CometHashAggregate (32) + +- CometColumnarExchange (31) + +- RowToColumnar (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.item (4) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometHashAggregate (20) + : +- CometColumnarExchange (19) + : +- RowToColumnar (18) + : +- * HashAggregate (17) + : +- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * ColumnarToRow (13) + : : +- CometFilter (12) + : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : +- ReusedExchange (14) + +- ReusedExchange (26) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -92,7 +96,7 @@ Condition : isnotnull(cs_item_sk#7) (13) ColumnarToRow [codegen id : 3] Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -(14) ReusedExchange [Reuses operator id: 34] +(14) ReusedExchange [Reuses operator id: 38] Output [1]: [d_date_sk#11] (15) BroadcastHashJoin [codegen id : 3] @@ -112,96 +116,104 @@ Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] Aggregate Attributes [2]: [sum#12, count#13] Results [3]: [cs_item_sk#7, sum#14, count#15] -(18) Exchange +(18) RowToColumnar Input [3]: [cs_item_sk#7, sum#14, count#15] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(19) HashAggregate [codegen id : 4] +(19) CometColumnarExchange +Input [3]: [cs_item_sk#7, sum#14, count#15] +Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(20) CometHashAggregate Input [3]: [cs_item_sk#7, sum#14, count#15] Keys [1]: [cs_item_sk#7] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#16] -Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] -(20) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) +(21) CometFilter +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) + +(22) ColumnarToRow [codegen id : 4] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -(21) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] +(23) BroadcastExchange +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] Right keys [1]: [cs_item_sk#7] Join type: Inner -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#16) -(23) Project [codegen id : 6] +(25) Project [codegen id : 6] Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -(24) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#18] +(26) ReusedExchange [Reuses operator id: 38] +Output [1]: [d_date_sk#17] -(25) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(28) Project [codegen id : 6] Output [1]: [cs_ext_discount_amt#2] -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#18] +Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#17] -(27) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [1]: [cs_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#19] -Results [1]: [sum#20] +Aggregate Attributes [1]: [sum#18] +Results [1]: [sum#19] -(28) Exchange -Input [1]: [sum#20] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(30) RowToColumnar +Input [1]: [sum#19] -(29) HashAggregate [codegen id : 7] -Input [1]: [sum#20] +(31) CometColumnarExchange +Input [1]: [sum#19] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(32) CometHashAggregate +Input [1]: [sum#19] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#21] -Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#21,17,2) AS excess discount amount#22] + +(33) ColumnarToRow [codegen id : 7] +Input [1]: [excess discount amount#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (34) -+- * ColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.date_dim (30) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#23] +Output [2]: [d_date_sk#17, d_date#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [d_date_sk#18, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) +(35) CometFilter +Input [2]: [d_date_sk#17, d_date#21] +Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 2000-01-27)) AND (d_date#21 <= 2000-04-26)) AND isnotnull(d_date_sk#17)) -(32) CometProject -Input [2]: [d_date_sk#18, d_date#23] -Arguments: [d_date_sk#18], [d_date_sk#18] +(36) CometProject +Input [2]: [d_date_sk#17, d_date#21] +Arguments: [d_date_sk#17], [d_date_sk#17] -(33) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] +(37) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#17] -(34) BroadcastExchange -Input [1]: [d_date_sk#18] +(38) BroadcastExchange +Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index 146a33fdd2..035779c3f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -1,52 +1,56 @@ WholeStageCodegen (7) - HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (6) - HashAggregate [cs_ext_discount_amt] [sum,sum] - Project [cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_discount_amt,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] - Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_ext_discount_amt] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometHashAggregate [sum] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [cs_ext_discount_amt] [sum,sum] + Project [cs_ext_discount_amt] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_discount_amt,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] + Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_ext_discount_amt] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #4 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometFilter [(1.3 * avg(cs_ext_discount_amt))] + CometHashAggregate [cs_item_sk,sum,count] + CometColumnarExchange [cs_item_sk] #5 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] + Project [cs_item_sk,cs_ext_discount_amt] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] - InputAdapter - Exchange [cs_item_sk] #5 - WholeStageCodegen (3) - HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] - Project [cs_item_sk,cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index d7dda8078a..3249f9fa01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -1,67 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * ColumnarToRow (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * ColumnarToRow (46) - : : : +- CometFilter (45) - : : : +- CometScan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +* ColumnarToRow (67) ++- CometTakeOrderedAndProject (66) + +- CometHashAggregate (65) + +- CometColumnarExchange (64) + +- CometHashAggregate (63) + +- CometUnion (62) + :- CometHashAggregate (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- CometHashAggregate (45) + : +- CometColumnarExchange (44) + : +- RowToColumnar (43) + : +- * HashAggregate (42) + : +- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (32) + : : : : +- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) + : : : +- ReusedExchange (33) + : : +- ReusedExchange (36) + : +- ReusedExchange (39) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) + +- RowToColumnar (59) + +- * HashAggregate (58) + +- * Project (57) + +- * BroadcastHashJoin Inner BuildRight (56) + :- * Project (54) + : +- * BroadcastHashJoin Inner BuildRight (53) + : :- * Project (51) + : : +- * BroadcastHashJoin Inner BuildRight (50) + : : :- * ColumnarToRow (48) + : : : +- CometFilter (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- ReusedExchange (49) + : +- ReusedExchange (52) + +- ReusedExchange (55) (unknown) Scan parquet spark_catalog.default.store_sales @@ -79,7 +83,7 @@ Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 68] +(4) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -187,219 +191,224 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#13] Results [2]: [i_manufact_id#10, sum#14] -(27) Exchange +(27) RowToColumnar Input [2]: [i_manufact_id#10, sum#14] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) HashAggregate [codegen id : 6] +(28) CometColumnarExchange +Input [2]: [i_manufact_id#10, sum#14] +Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(29) CometHashAggregate Input [2]: [i_manufact_id#10, sum#14] Keys [1]: [i_manufact_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Output [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) +(31) CometFilter +Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] +Condition : (isnotnull(cs_bill_addr_sk#15) AND isnotnull(cs_item_sk#16)) -(31) ColumnarToRow [codegen id : 11] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +(32) ColumnarToRow [codegen id : 10] +Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -(32) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#22] +(33) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#20] -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] +(34) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(34) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] +(35) Project [codegen id : 10] +Output [3]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17] +Input [5]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#20] -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#23] +(36) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#21] -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#23] +(37) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_bill_addr_sk#15] +Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(37) Project [codegen id : 11] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] +(38) Project [codegen id : 10] +Output [2]: [cs_item_sk#16, cs_ext_sales_price#17] +Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, ca_address_sk#21] -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#24, i_manufact_id#25] +(39) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#22, i_manufact_id#23] -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#24] +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_item_sk#16] +Right keys [1]: [i_item_sk#22] Join type: Inner Join condition: None -(40) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#19, i_manufact_id#25] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_manufact_id#25] +(41) Project [codegen id : 10] +Output [2]: [cs_ext_sales_price#17, i_manufact_id#23] +Input [4]: [cs_item_sk#16, cs_ext_sales_price#17, i_item_sk#22, i_manufact_id#23] + +(42) HashAggregate [codegen id : 10] +Input [2]: [cs_ext_sales_price#17, i_manufact_id#23] +Keys [1]: [i_manufact_id#23] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#17))] +Aggregate Attributes [1]: [sum#24] +Results [2]: [i_manufact_id#23, sum#25] -(41) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#19, i_manufact_id#25] -Keys [1]: [i_manufact_id#25] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_manufact_id#25, sum#27] +(43) RowToColumnar +Input [2]: [i_manufact_id#23, sum#25] -(42) Exchange -Input [2]: [i_manufact_id#25, sum#27] -Arguments: hashpartitioning(i_manufact_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(44) CometColumnarExchange +Input [2]: [i_manufact_id#23, sum#25] +Arguments: hashpartitioning(i_manufact_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#25, sum#27] -Keys [1]: [i_manufact_id#25] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] -Results [2]: [i_manufact_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] +(45) CometHashAggregate +Input [2]: [i_manufact_id#23, sum#25] +Keys [1]: [i_manufact_id#23] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#17))] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) +(47) CometFilter +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) -(46) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +(48) ColumnarToRow [codegen id : 15] +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -(47) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#35] +(49) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#31] -(48) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#35] +(50) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#29] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(49) Project [codegen id : 17] -Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] +(51) Project [codegen id : 15] +Output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] +Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] -(50) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#36] +(52) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#32] -(51) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#31] -Right keys [1]: [ca_address_sk#36] +(53) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_bill_addr_sk#27] +Right keys [1]: [ca_address_sk#32] Join type: Inner Join condition: None -(52) Project [codegen id : 17] -Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] +(54) Project [codegen id : 15] +Output [2]: [ws_item_sk#26, ws_ext_sales_price#28] +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] -(53) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#37, i_manufact_id#38] +(55) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#33, i_manufact_id#34] -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#30] -Right keys [1]: [i_item_sk#37] +(56) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#33] Join type: Inner Join condition: None -(55) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#32, i_manufact_id#38] -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_manufact_id#38] +(57) Project [codegen id : 15] +Output [2]: [ws_ext_sales_price#28, i_manufact_id#34] +Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] -(56) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#32, i_manufact_id#38] -Keys [1]: [i_manufact_id#38] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum#39] -Results [2]: [i_manufact_id#38, sum#40] +(58) HashAggregate [codegen id : 15] +Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] +Keys [1]: [i_manufact_id#34] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] +Aggregate Attributes [1]: [sum#35] +Results [2]: [i_manufact_id#34, sum#36] -(57) Exchange -Input [2]: [i_manufact_id#38, sum#40] -Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(59) RowToColumnar +Input [2]: [i_manufact_id#34, sum#36] -(58) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#38, sum#40] -Keys [1]: [i_manufact_id#38] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] -Results [2]: [i_manufact_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] +(60) CometColumnarExchange +Input [2]: [i_manufact_id#34, sum#36] +Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(59) Union +(61) CometHashAggregate +Input [2]: [i_manufact_id#34, sum#36] +Keys [1]: [i_manufact_id#34] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -(60) HashAggregate [codegen id : 19] -Input [2]: [i_manufact_id#10, total_sales#16] +(62) CometUnion +Child 0 Input [2]: [i_manufact_id#10, total_sales#37] +Child 1 Input [2]: [i_manufact_id#23, total_sales#38] +Child 2 Input [2]: [i_manufact_id#34, total_sales#39] + +(63) CometHashAggregate +Input [2]: [i_manufact_id#10, total_sales#37] Keys [1]: [i_manufact_id#10] -Functions [1]: [partial_sum(total_sales#16)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [3]: [i_manufact_id#10, sum#45, isEmpty#46] +Functions [1]: [partial_sum(total_sales#37)] -(61) Exchange -Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(64) CometColumnarExchange +Input [3]: [i_manufact_id#10, sum#40, isEmpty#41] +Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(62) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] +(65) CometHashAggregate +Input [3]: [i_manufact_id#10, sum#40, isEmpty#41] Keys [1]: [i_manufact_id#10] -Functions [1]: [sum(total_sales#16)] -Aggregate Attributes [1]: [sum(total_sales#16)#47] -Results [2]: [i_manufact_id#10, sum(total_sales#16)#47 AS total_sales#48] +Functions [1]: [sum(total_sales#37)] + +(66) CometTakeOrderedAndProject +Input [2]: [i_manufact_id#10, total_sales#42] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#42 ASC NULLS FIRST], output=[i_manufact_id#10,total_sales#42]), 100, [total_sales#42 ASC NULLS FIRST], [i_manufact_id#10, total_sales#42] -(63) TakeOrderedAndProject -Input [2]: [i_manufact_id#10, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#10, total_sales#48] +(67) ColumnarToRow [codegen id : 16] +Input [2]: [i_manufact_id#10, total_sales#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#49, d_moy#50] +Output [3]: [d_date_sk#6, d_year#43, d_moy#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] ReadSchema: struct -(65) CometFilter -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 5)) AND isnotnull(d_date_sk#6)) +(69) CometFilter +Input [3]: [d_date_sk#6, d_year#43, d_moy#44] +Condition : ((((isnotnull(d_year#43) AND isnotnull(d_moy#44)) AND (d_year#43 = 1998)) AND (d_moy#44 = 5)) AND isnotnull(d_date_sk#6)) -(66) CometProject -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +(70) CometProject +Input [3]: [d_date_sk#6, d_year#43, d_moy#44] Arguments: [d_date_sk#6], [d_date_sk#6] -(67) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 4ab82379ff..9dd28b41bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -1,105 +1,101 @@ -TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (20) - HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_manufact_id] #1 - WholeStageCodegen (19) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #2 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sales,i_manufact_id] + CometHashAggregate [i_manufact_id,sum,isEmpty] + CometColumnarExchange [i_manufact_id] #1 + CometHashAggregate [i_manufact_id,total_sales] + CometUnion + CometHashAggregate [i_manufact_id,sum] + CometColumnarExchange [i_manufact_id] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_manufact_id,i_manufact_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #6 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_manufact_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (12) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #7 - WholeStageCodegen (11) - HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [i_manufact_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] + CometHashAggregate [i_manufact_id,sum] + CometColumnarExchange [i_manufact_id] #7 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [ca_address_sk] #4 + ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 - WholeStageCodegen (18) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_manufact_id] #8 - WholeStageCodegen (17) - HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_manufact_id] #5 + CometHashAggregate [i_manufact_id,sum] + CometColumnarExchange [i_manufact_id] #8 + RowToColumnar + WholeStageCodegen (15) + HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [ca_address_sk] #4 + ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt index 64f2b3c312..d2ff5ebe98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt @@ -1,36 +1,40 @@ == Physical Plan == -* Sort (32) -+- Exchange (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (24) - : +- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (28) - +- * ColumnarToRow (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.customer (25) +* ColumnarToRow (36) ++- CometSort (35) + +- CometColumnarExchange (34) + +- RowToColumnar (33) + +- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * ColumnarToRow (26) + : +- CometFilter (25) + : +- CometHashAggregate (24) + : +- CometColumnarExchange (23) + : +- RowToColumnar (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (30) + +- * ColumnarToRow (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.customer (27) (unknown) Scan parquet spark_catalog.default.store_sales @@ -48,7 +52,7 @@ Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnu (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 37] +(4) ReusedExchange [Reuses operator id: 41] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -132,86 +136,96 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#14] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) Exchange +(22) RowToColumnar Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(23) HashAggregate [codegen id : 6] +(23) CometColumnarExchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(24) CometHashAggregate Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(24) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) +(25) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] +Condition : ((cnt#16 >= 15) AND (cnt#16 <= 20)) + +(26) ColumnarToRow [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] (unknown) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Output [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) +(28) CometFilter +Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Condition : isnotnull(c_customer_sk#17) -(27) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(29) ColumnarToRow [codegen id : 5] +Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] -(28) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(30) BroadcastExchange +Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#17] Join type: Inner Join condition: None -(30) Project [codegen id : 6] -Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(32) Project [codegen id : 6] +Output [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16, c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] + +(33) RowToColumnar +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] + +(34) CometColumnarExchange +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Arguments: rangepartitioning(c_last_name#20 ASC NULLS FIRST, c_first_name#19 ASC NULLS FIRST, c_salutation#18 ASC NULLS FIRST, c_preferred_cust_flag#21 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(31) Exchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(35) CometSort +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Arguments: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16], [c_last_name#20 ASC NULLS FIRST, c_first_name#19 ASC NULLS FIRST, c_salutation#18 ASC NULLS FIRST, c_preferred_cust_flag#21 DESC NULLS LAST] -(32) Sort [codegen id : 7] -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST], true, 0 +(36) ColumnarToRow [codegen id : 7] +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (41) ++- * ColumnarToRow (40) + +- CometProject (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.date_dim (37) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +Output [3]: [d_date_sk#7, d_year#22, d_dom#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] -Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(38) CometFilter +Input [3]: [d_date_sk#7, d_year#22, d_dom#23] +Condition : (((((d_dom#23 >= 1) AND (d_dom#23 <= 3)) OR ((d_dom#23 >= 25) AND (d_dom#23 <= 28))) AND d_year#22 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(35) CometProject -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +(39) CometProject +Input [3]: [d_date_sk#7, d_year#22, d_dom#23] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(40) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(37) BroadcastExchange +(41) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index 80405a784d..eed22a0307 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -1,56 +1,60 @@ WholeStageCodegen (7) - Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - InputAdapter - Exchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] + CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + RowToColumnar + WholeStageCodegen (6) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index de7514efce..4d8be2c63a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +* ColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * Filter (25) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_demographics (33) (unknown) Scan parquet spark_catalog.default.customer @@ -67,7 +69,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) ReusedExchange [Reuses operator id: 47] +(6) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#9] (7) BroadcastHashJoin [codegen id : 2] @@ -100,7 +102,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(13) ReusedExchange [Reuses operator id: 47] +(13) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#13] (14) BroadcastHashJoin [codegen id : 4] @@ -133,7 +135,7 @@ ReadSchema: struct (19) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -(20) ReusedExchange [Reuses operator id: 47] +(20) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 6] @@ -227,50 +229,54 @@ Functions [10]: [partial_count(1), partial_min(cd_dep_count#23), partial_max(cd_ Aggregate Attributes [13]: [count#26, min#27, max#28, sum#29, count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38] Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] -(40) Exchange +(40) RowToColumnar Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) HashAggregate [codegen id : 10] +(41) CometColumnarExchange +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] +Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(42) CometHashAggregate Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Functions [10]: [count(1), min(cd_dep_count#23), max(cd_dep_count#23), avg(cd_dep_count#23), min(cd_dep_employed_count#24), max(cd_dep_employed_count#24), avg(cd_dep_employed_count#24), min(cd_dep_college_count#25), max(cd_dep_college_count#25), avg(cd_dep_college_count#25)] -Aggregate Attributes [10]: [count(1)#52, min(cd_dep_count#23)#53, max(cd_dep_count#23)#54, avg(cd_dep_count#23)#55, min(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, avg(cd_dep_employed_count#24)#58, min(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, avg(cd_dep_college_count#25)#61] -Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, count(1)#52 AS cnt1#62, min(cd_dep_count#23)#53 AS min(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, avg(cd_dep_count#23)#55 AS avg(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, min(cd_dep_employed_count#24)#56 AS min(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, avg(cd_dep_employed_count#24)#58 AS avg(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, min(cd_dep_college_count#25)#59 AS min(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, avg(cd_dep_college_count#25)#61 AS avg(cd_dep_college_count)#73, cd_dep_count#23] -(42) TakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73, cd_dep_count#23] -Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73] +(43) CometTakeOrderedAndProject +Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#52, min(cd_dep_count)#53, max(cd_dep_count)#54, avg(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, min(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, avg(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, min(cd_dep_college_count)#61, max(cd_dep_college_count)#62, avg(cd_dep_college_count)#63, cd_dep_count#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#19 ASC NULLS FIRST,cd_gender#21 ASC NULLS FIRST,cd_marital_status#22 ASC NULLS FIRST,cd_dep_count#23 ASC NULLS FIRST,cd_dep_employed_count#24 ASC NULLS FIRST,cd_dep_college_count#25 ASC NULLS FIRST], output=[ca_state#19,cd_gender#21,cd_marital_status#22,cnt1#52,min(cd_dep_count)#53,max(cd_dep_count)#54,avg(cd_dep_count)#55,cd_dep_employed_count#24,cnt2#56,min(cd_dep_employed_count)#57,max(cd_dep_employed_count)#58,avg(cd_dep_employed_count)#59,cd_dep_college_count#25,cnt3#60,min(cd_dep_college_count)#61,max(cd_dep_college_count)#62,avg(cd_dep_college_count)#63]), 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#52, min(cd_dep_count)#53, max(cd_dep_count)#54, avg(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, min(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, avg(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, min(cd_dep_college_count)#61, max(cd_dep_college_count)#62, avg(cd_dep_college_count)#63] + +(44) ColumnarToRow [codegen id : 10] +Input [17]: [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#52, min(cd_dep_count)#53, max(cd_dep_count)#54, avg(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, min(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, avg(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, min(cd_dep_college_count)#61, max(cd_dep_college_count)#62, avg(cd_dep_college_count)#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#74, d_qoy#75] +Output [3]: [d_date_sk#9, d_year#64, d_qoy#65] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] -Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) +(46) CometFilter +Input [3]: [d_date_sk#9, d_year#64, d_qoy#65] +Condition : ((((isnotnull(d_year#64) AND isnotnull(d_qoy#65)) AND (d_year#64 = 2002)) AND (d_qoy#65 < 4)) AND isnotnull(d_date_sk#9)) -(45) CometProject -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +(47) CometProject +Input [3]: [d_date_sk#9, d_year#64, d_qoy#65] Arguments: [d_date_sk#9], [d_date_sk#9] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index ea0ef274ea..c2b0d7dde1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -1,74 +1,76 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - InputAdapter - Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow +WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt index bb2a1b1a5c..a71f17fb59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt @@ -1,32 +1,34 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * Project (27) - +- Window (26) - +- * Sort (25) - +- Exchange (24) - +- * HashAggregate (23) - +- Exchange (22) - +- * HashAggregate (21) - +- * Expand (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store (13) +TakeOrderedAndProject (30) ++- * Project (29) + +- Window (28) + +- * ColumnarToRow (27) + +- CometSort (26) + +- CometColumnarExchange (25) + +- CometHashAggregate (24) + +- CometColumnarExchange (23) + +- RowToColumnar (22) + +- * HashAggregate (21) + +- * Expand (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometProject (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.store (13) (unknown) Scan parquet spark_catalog.default.store_sales @@ -44,7 +46,7 @@ Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 33] +(4) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -128,66 +130,70 @@ Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(Unscale Aggregate Attributes [2]: [sum#16, sum#17] Results [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] -(22) Exchange +(22) RowToColumnar Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] -Arguments: hashpartitioning(i_category#13, i_class#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(23) HashAggregate [codegen id : 5] +(23) CometColumnarExchange +Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] +Arguments: hashpartitioning(i_category#13, i_class#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(24) CometHashAggregate Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledValue(ss_ext_sales_price#3))#21] -Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS _w0#24, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS _w1#25, CASE WHEN (cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint) = 0) THEN i_category#13 END AS _w2#26] -(24) Exchange -Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -Arguments: hashpartitioning(_w1#25, _w2#26, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(25) CometColumnarExchange +Input [7]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24] +Arguments: hashpartitioning(_w1#23, _w2#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(26) CometSort +Input [7]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24] +Arguments: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24], [_w1#23 ASC NULLS FIRST, _w2#24 ASC NULLS FIRST, _w0#22 ASC NULLS FIRST] -(25) Sort [codegen id : 6] -Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -Arguments: [_w1#25 ASC NULLS FIRST, _w2#26 ASC NULLS FIRST, _w0#24 ASC NULLS FIRST], false, 0 +(27) ColumnarToRow [codegen id : 5] +Input [7]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24] -(26) Window -Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -Arguments: [rank(_w0#24) windowspecdefinition(_w1#25, _w2#26, _w0#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#27], [_w1#25, _w2#26], [_w0#24 ASC NULLS FIRST] +(28) Window +Input [7]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24] +Arguments: [rank(_w0#22) windowspecdefinition(_w1#23, _w2#24, _w0#22 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#25], [_w1#23, _w2#24], [_w0#22 ASC NULLS FIRST] -(27) Project [codegen id : 7] -Output [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] -Input [8]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26, rank_within_parent#27] +(29) Project [codegen id : 6] +Output [5]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, rank_within_parent#25] +Input [8]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24, rank_within_parent#25] -(28) TakeOrderedAndProject -Input [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] -Arguments: 100, [lochierarchy#23 DESC NULLS LAST, CASE WHEN (lochierarchy#23 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#27 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] +(30) TakeOrderedAndProject +Input [5]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, rank_within_parent#25] +Arguments: 100, [lochierarchy#21 DESC NULLS LAST, CASE WHEN (lochierarchy#21 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#25 ASC NULLS FIRST], [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, rank_within_parent#25] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#28] +Output [2]: [d_date_sk#7, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(30) CometFilter -Input [2]: [d_date_sk#7, d_year#28] -Condition : ((isnotnull(d_year#28) AND (d_year#28 = 2001)) AND isnotnull(d_date_sk#7)) +(32) CometFilter +Input [2]: [d_date_sk#7, d_year#26] +Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#7)) -(31) CometProject -Input [2]: [d_date_sk#7, d_year#28] +(33) CometProject +Input [2]: [d_date_sk#7, d_year#26] Arguments: [d_date_sk#7], [d_date_sk#7] -(32) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(33) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index 7eeb607c3b..e9e6131a9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -1,16 +1,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (7) + WholeStageCodegen (6) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (6) - Sort [_w1,_w2,_w0] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - InputAdapter - Exchange [i_category,i_class,spark_grouping_id] #2 + CometSort [_w1,_w2,_w0] + CometColumnarExchange [_w1,_w2] #1 + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum] + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + RowToColumnar WholeStageCodegen (4) HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt index d13ff264cd..06405034bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt @@ -1,29 +1,31 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * ColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan parquet spark_catalog.default.catalog_sales (16) +* ColumnarToRow (27) ++- CometTakeOrderedAndProject (26) + +- CometHashAggregate (25) + +- CometColumnarExchange (24) + +- RowToColumnar (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildLeft (20) + :- BroadcastExchange (15) + : +- * Project (14) + : +- * BroadcastHashJoin Inner BuildRight (13) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (9) + : : +- * ColumnarToRow (8) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.inventory (5) + : +- ReusedExchange (12) + +- * ColumnarToRow (19) + +- CometProject (18) + +- CometFilter (17) + +- CometScan parquet spark_catalog.default.catalog_sales (16) (unknown) Scan parquet spark_catalog.default.item @@ -77,7 +79,7 @@ Join condition: None Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_item_sk#6, inv_date_sk#8] -(12) ReusedExchange [Reuses operator id: 30] +(12) ReusedExchange [Reuses operator id: 32] Output [1]: [d_date_sk#10] (13) BroadcastHashJoin [codegen id : 3] @@ -129,29 +131,33 @@ Functions: [] Aggregate Attributes: [] Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(23) Exchange +(23) RowToColumnar Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) HashAggregate [codegen id : 5] +(24) CometColumnarExchange +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(25) CometHashAggregate Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(25) TakeOrderedAndProject +(26) CometTakeOrderedAndProject +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] + +(27) ColumnarToRow [codegen id : 5] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (30) -+- * ColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.date_dim (26) +BroadcastExchange (32) ++- * ColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan parquet spark_catalog.default.date_dim (28) (unknown) Scan parquet spark_catalog.default.date_dim @@ -161,18 +167,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter +(29) CometFilter Input [2]: [d_date_sk#10, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-01)) AND (d_date#13 <= 2000-04-01)) AND isnotnull(d_date_sk#10)) -(28) CometProject +(30) CometProject Input [2]: [d_date_sk#10, d_date#13] Arguments: [d_date_sk#10], [d_date_sk#10] -(29) ColumnarToRow [codegen id : 1] +(31) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(30) BroadcastExchange +(32) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt index 65bb06348b..3f84fa0598 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt @@ -1,44 +1,46 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - WholeStageCodegen (5) - HashAggregate [i_item_id,i_item_desc,i_current_price] - InputAdapter - Exchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] +WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,cs_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (3) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - ColumnarToRow - InputAdapter - CometProject [cs_item_sk] - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedExchange [d_date_sk] #4 + ColumnarToRow + InputAdapter + CometProject [cs_item_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index 87d960592d..d42b2f1bf6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -1,51 +1,59 @@ == Physical Plan == -* HashAggregate (47) -+- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin LeftSemi BuildRight (43) - :- * BroadcastHashJoin LeftSemi BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer (7) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- BroadcastExchange (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * ColumnarToRow (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.web_sales (30) - : +- ReusedExchange (33) - +- ReusedExchange (36) +* ColumnarToRow (55) ++- CometHashAggregate (54) + +- CometColumnarExchange (53) + +- RowToColumnar (52) + +- * HashAggregate (51) + +- * Project (50) + +- * BroadcastHashJoin LeftSemi BuildRight (49) + :- * BroadcastHashJoin LeftSemi BuildRight (33) + : :- * ColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometColumnarExchange (15) + : : +- RowToColumnar (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer (7) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometHashAggregate (30) + : +- CometColumnarExchange (29) + : +- RowToColumnar (28) + : +- * HashAggregate (27) + : +- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * ColumnarToRow (20) + : : : +- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (21) + : +- ReusedExchange (24) + +- BroadcastExchange (48) + +- * ColumnarToRow (47) + +- CometHashAggregate (46) + +- CometColumnarExchange (45) + +- RowToColumnar (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * ColumnarToRow (36) + : : +- CometFilter (35) + : : +- CometScan parquet spark_catalog.default.web_sales (34) + : +- ReusedExchange (37) + +- ReusedExchange (40) (unknown) Scan parquet spark_catalog.default.store_sales @@ -63,7 +71,7 @@ Condition : isnotnull(ss_customer_sk#1) (3) ColumnarToRow [codegen id : 3] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -(4) ReusedExchange [Reuses operator id: 52] +(4) ReusedExchange [Reuses operator id: 60] Output [2]: [d_date_sk#4, d_date#5] (5) BroadcastHashJoin [codegen id : 3] @@ -111,16 +119,20 @@ Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#8, c_first_name#7, d_date#5] -(14) Exchange +(14) RowToColumnar Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 12] +(15) CometColumnarExchange +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [3]: [c_last_name#8, c_first_name#7, d_date#5] Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] + +(17) ColumnarToRow [codegen id : 12] +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] @@ -130,62 +142,66 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(17) CometFilter +(19) CometFilter Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] Condition : isnotnull(cs_bill_customer_sk#9) -(18) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 6] Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] -(19) ReusedExchange [Reuses operator id: 52] +(21) ReusedExchange [Reuses operator id: 60] Output [2]: [d_date_sk#12, d_date#13] -(20) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(21) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [2]: [cs_bill_customer_sk#9, d_date#13] Input [4]: [cs_bill_customer_sk#9, cs_sold_date_sk#10, d_date_sk#12, d_date#13] -(22) ReusedExchange [Reuses operator id: 10] +(24) ReusedExchange [Reuses operator id: 10] Output [3]: [c_customer_sk#14, c_first_name#15, c_last_name#16] -(23) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_bill_customer_sk#9] Right keys [1]: [c_customer_sk#14] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [3]: [c_last_name#16, c_first_name#15, d_date#13] Input [5]: [cs_bill_customer_sk#9, d_date#13, c_customer_sk#14, c_first_name#15, c_last_name#16] -(25) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 6] Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(26) Exchange +(28) RowToColumnar Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(27) HashAggregate [codegen id : 7] +(29) CometColumnarExchange +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(30) CometHashAggregate Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(28) BroadcastExchange +(31) ColumnarToRow [codegen id : 7] +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] + +(32) BroadcastExchange Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 12] +(33) BroadcastHashJoin [codegen id : 12] Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#16, ), isnull(c_last_name#16), coalesce(c_first_name#15, ), isnull(c_first_name#15), coalesce(d_date#13, 1970-01-01), isnull(d_date#13)] Join type: LeftSemi @@ -199,123 +215,131 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#18), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(31) CometFilter +(35) CometFilter Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] Condition : isnotnull(ws_bill_customer_sk#17) -(32) ColumnarToRow [codegen id : 10] +(36) ColumnarToRow [codegen id : 10] Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] -(33) ReusedExchange [Reuses operator id: 52] +(37) ReusedExchange [Reuses operator id: 60] Output [2]: [d_date_sk#20, d_date#21] -(34) BroadcastHashJoin [codegen id : 10] +(38) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#18] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 10] +(39) Project [codegen id : 10] Output [2]: [ws_bill_customer_sk#17, d_date#21] Input [4]: [ws_bill_customer_sk#17, ws_sold_date_sk#18, d_date_sk#20, d_date#21] -(36) ReusedExchange [Reuses operator id: 10] +(40) ReusedExchange [Reuses operator id: 10] Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] -(37) BroadcastHashJoin [codegen id : 10] +(41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_bill_customer_sk#17] Right keys [1]: [c_customer_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(42) Project [codegen id : 10] Output [3]: [c_last_name#24, c_first_name#23, d_date#21] Input [5]: [ws_bill_customer_sk#17, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] -(39) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 10] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(40) Exchange +(44) RowToColumnar +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] + +(45) CometColumnarExchange Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) HashAggregate [codegen id : 11] +(46) CometHashAggregate Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(42) BroadcastExchange +(47) ColumnarToRow [codegen id : 11] +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] + +(48) BroadcastExchange Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 12] +(49) BroadcastHashJoin [codegen id : 12] Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] Join type: LeftSemi Join condition: None -(44) Project [codegen id : 12] +(50) Project [codegen id : 12] Output: [] Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -(45) HashAggregate [codegen id : 12] +(51) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#25] Results [1]: [count#26] -(46) Exchange +(52) RowToColumnar Input [1]: [count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(47) HashAggregate [codegen id : 13] +(53) CometColumnarExchange +Input [1]: [count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(54) CometHashAggregate Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#27] -Results [1]: [count(1)#27 AS count(1)#28] + +(55) ColumnarToRow [codegen id : 13] +Input [1]: [count(1)#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (60) ++- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) +(57) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#28] +Condition : (((isnotnull(d_month_seq#28) AND (d_month_seq#28 >= 1200)) AND (d_month_seq#28 <= 1211)) AND isnotnull(d_date_sk#4)) -(50) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +(58) CometProject +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#28] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(59) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(52) BroadcastExchange +(60) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 +Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 34 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index 315afe6602..28771f9d42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -1,81 +1,89 @@ WholeStageCodegen (13) - HashAggregate [count] [count(1),count(1),count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (12) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) + CometHashAggregate [count] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (12) + HashAggregate [count,count] + Project + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #6 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #8 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index e10ff3340d..11adf85554 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -1,48 +1,54 @@ == Physical Plan == -* Sort (44) -+- Exchange (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (23) - : +- * Filter (22) - : +- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (41) - +- * Project (40) - +- * Filter (39) - +- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * ColumnarToRow (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (27) - : +- ReusedExchange (30) - +- ReusedExchange (33) +* ColumnarToRow (50) ++- CometSort (49) + +- CometColumnarExchange (48) + +- RowToColumnar (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (25) + : +- * Filter (24) + : +- * HashAggregate (23) + : +- * ColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- RowToColumnar (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.warehouse (10) + : +- ReusedExchange (16) + +- BroadcastExchange (45) + +- * Project (44) + +- * Filter (43) + +- * HashAggregate (42) + +- * ColumnarToRow (41) + +- CometColumnarExchange (40) + +- RowToColumnar (39) + +- * HashAggregate (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * ColumnarToRow (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.inventory (26) + : : +- ReusedExchange (29) + : +- ReusedExchange (32) + +- ReusedExchange (35) (unknown) Scan parquet spark_catalog.default.inventory @@ -116,7 +122,7 @@ Join condition: None Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -(16) ReusedExchange [Reuses operator id: 49] +(16) ReusedExchange [Reuses operator id: 55] Output [2]: [d_date_sk#9, d_moy#10] (17) BroadcastHashJoin [codegen id : 4] @@ -136,22 +142,28 @@ Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), par Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(20) Exchange +(20) RowToColumnar Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) HashAggregate [codegen id : 10] +(21) CometColumnarExchange +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) ColumnarToRow [codegen id : 10] +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] + +(23) HashAggregate [codegen id : 10] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] -(22) Filter [codegen id : 10] +(24) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END -(23) Project [codegen id : 10] +(25) Project [codegen id : 10] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] @@ -163,104 +175,116 @@ PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(25) CometFilter +(27) CometFilter Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) -(26) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 8] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -(27) ReusedExchange [Reuses operator id: 7] +(29) ReusedExchange [Reuses operator id: 7] Output [1]: [i_item_sk#31] -(28) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_item_sk#26] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(29) Project [codegen id : 8] +(31) Project [codegen id : 8] Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -(30) ReusedExchange [Reuses operator id: 13] +(32) ReusedExchange [Reuses operator id: 13] Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] -(31) BroadcastHashJoin [codegen id : 8] +(33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_warehouse_sk#27] Right keys [1]: [w_warehouse_sk#32] Join type: Inner Join condition: None -(32) Project [codegen id : 8] +(34) Project [codegen id : 8] Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -(33) ReusedExchange [Reuses operator id: 54] +(35) ReusedExchange [Reuses operator id: 60] Output [2]: [d_date_sk#34, d_moy#35] -(34) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(35) Project [codegen id : 8] +(37) Project [codegen id : 8] Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] -(36) HashAggregate [codegen id : 8] +(38) HashAggregate [codegen id : 8] Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -(37) Exchange +(39) RowToColumnar +Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] + +(40) CometColumnarExchange Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(38) HashAggregate [codegen id : 9] +(41) ColumnarToRow [codegen id : 9] +Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] + +(42) HashAggregate [codegen id : 9] Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] -(39) Filter [codegen id : 9] +(43) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END -(40) Project [codegen id : 9] +(44) Project [codegen id : 9] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -(41) BroadcastExchange +(45) BroadcastExchange Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(42) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] Join type: Inner Join condition: None -(43) Exchange +(47) RowToColumnar +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] + +(48) CometColumnarExchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(49) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] -(44) Sort [codegen id : 11] +(50) ColumnarToRow [codegen id : 11] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (55) ++- * ColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) (unknown) Scan parquet spark_catalog.default.date_dim @@ -270,27 +294,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(52) CometFilter Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) -(47) CometProject +(53) CometProject Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] -(48) ColumnarToRow [codegen id : 1] +(54) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#10] -(49) BroadcastExchange +(55) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +Subquery:2 Hosting operator id = 26 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 +BroadcastExchange (60) ++- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) (unknown) Scan parquet spark_catalog.default.date_dim @@ -300,18 +324,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter +(57) CometFilter Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) -(52) CometProject +(58) CometProject Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] -(53) ColumnarToRow [codegen id : 1] +(59) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_moy#35] -(54) BroadcastExchange +(60) BroadcastExchange Input [2]: [d_date_sk#34, d_moy#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 002266e76e..0dc072692c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -1,81 +1,87 @@ WholeStageCodegen (11) - Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] + ColumnarToRow InputAdapter - Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] + CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + RowToColumnar + WholeStageCodegen (10) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (9) + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [i_item_sk] #4 + InputAdapter + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index 98e8bc4642..cf500fbd99 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -1,48 +1,54 @@ == Physical Plan == -* Sort (44) -+- Exchange (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (23) - : +- * Filter (22) - : +- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (41) - +- * Project (40) - +- * Filter (39) - +- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * ColumnarToRow (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (27) - : +- ReusedExchange (30) - +- ReusedExchange (33) +* ColumnarToRow (50) ++- CometSort (49) + +- CometColumnarExchange (48) + +- RowToColumnar (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (25) + : +- * Filter (24) + : +- * HashAggregate (23) + : +- * ColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- RowToColumnar (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.warehouse (10) + : +- ReusedExchange (16) + +- BroadcastExchange (45) + +- * Project (44) + +- * Filter (43) + +- * HashAggregate (42) + +- * ColumnarToRow (41) + +- CometColumnarExchange (40) + +- RowToColumnar (39) + +- * HashAggregate (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * ColumnarToRow (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.inventory (26) + : : +- ReusedExchange (29) + : +- ReusedExchange (32) + +- ReusedExchange (35) (unknown) Scan parquet spark_catalog.default.inventory @@ -116,7 +122,7 @@ Join condition: None Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -(16) ReusedExchange [Reuses operator id: 49] +(16) ReusedExchange [Reuses operator id: 55] Output [2]: [d_date_sk#9, d_moy#10] (17) BroadcastHashJoin [codegen id : 4] @@ -136,22 +142,28 @@ Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), par Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(20) Exchange +(20) RowToColumnar Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) HashAggregate [codegen id : 10] +(21) CometColumnarExchange +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) ColumnarToRow [codegen id : 10] +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] + +(23) HashAggregate [codegen id : 10] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] -(22) Filter [codegen id : 10] +(24) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] Condition : (CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END AND CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.5) END) -(23) Project [codegen id : 10] +(25) Project [codegen id : 10] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] @@ -163,104 +175,116 @@ PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(25) CometFilter +(27) CometFilter Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) -(26) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 8] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -(27) ReusedExchange [Reuses operator id: 7] +(29) ReusedExchange [Reuses operator id: 7] Output [1]: [i_item_sk#31] -(28) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_item_sk#26] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(29) Project [codegen id : 8] +(31) Project [codegen id : 8] Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -(30) ReusedExchange [Reuses operator id: 13] +(32) ReusedExchange [Reuses operator id: 13] Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] -(31) BroadcastHashJoin [codegen id : 8] +(33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_warehouse_sk#27] Right keys [1]: [w_warehouse_sk#32] Join type: Inner Join condition: None -(32) Project [codegen id : 8] +(34) Project [codegen id : 8] Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -(33) ReusedExchange [Reuses operator id: 54] +(35) ReusedExchange [Reuses operator id: 60] Output [2]: [d_date_sk#34, d_moy#35] -(34) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(35) Project [codegen id : 8] +(37) Project [codegen id : 8] Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] -(36) HashAggregate [codegen id : 8] +(38) HashAggregate [codegen id : 8] Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -(37) Exchange +(39) RowToColumnar +Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] + +(40) CometColumnarExchange Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(38) HashAggregate [codegen id : 9] +(41) ColumnarToRow [codegen id : 9] +Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] + +(42) HashAggregate [codegen id : 9] Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] -(39) Filter [codegen id : 9] +(43) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END -(40) Project [codegen id : 9] +(44) Project [codegen id : 9] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -(41) BroadcastExchange +(45) BroadcastExchange Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(42) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] Join type: Inner Join condition: None -(43) Exchange +(47) RowToColumnar +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] + +(48) CometColumnarExchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(49) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] -(44) Sort [codegen id : 11] +(50) ColumnarToRow [codegen id : 11] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (55) ++- * ColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) (unknown) Scan parquet spark_catalog.default.date_dim @@ -270,27 +294,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(52) CometFilter Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) -(47) CometProject +(53) CometProject Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] -(48) ColumnarToRow [codegen id : 1] +(54) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#10] -(49) BroadcastExchange +(55) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +Subquery:2 Hosting operator id = 26 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 +BroadcastExchange (60) ++- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) (unknown) Scan parquet spark_catalog.default.date_dim @@ -300,18 +324,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter +(57) CometFilter Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) -(52) CometProject +(58) CometProject Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] -(53) ColumnarToRow [codegen id : 1] +(59) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_moy#35] -(54) BroadcastExchange +(60) BroadcastExchange Input [2]: [d_date_sk#34, d_moy#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 002266e76e..0dc072692c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -1,81 +1,87 @@ WholeStageCodegen (11) - Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] + ColumnarToRow InputAdapter - Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] + CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + RowToColumnar + WholeStageCodegen (10) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (9) + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [i_item_sk] #4 + InputAdapter + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index 9b2bb49916..092603e845 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -1,112 +1,124 @@ == Physical Plan == -TakeOrderedAndProject (108) -+- * Project (107) - +- * BroadcastHashJoin Inner BuildRight (106) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * Project (52) - : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : :- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Filter (16) - : : : : : +- * HashAggregate (15) - : : : : : +- Exchange (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (10) - : : : : +- BroadcastExchange (32) - : : : : +- * HashAggregate (31) - : : : : +- Exchange (30) - : : : : +- * HashAggregate (29) - : : : : +- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * Project (25) - : : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : : :- * ColumnarToRow (19) - : : : : : : +- CometFilter (18) - : : : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : : : +- BroadcastExchange (23) - : : : : : +- * ColumnarToRow (22) - : : : : : +- CometFilter (21) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : : : +- ReusedExchange (26) - : : : +- BroadcastExchange (50) - : : : +- * Filter (49) - : : : +- * HashAggregate (48) - : : : +- Exchange (47) - : : : +- * HashAggregate (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * ColumnarToRow (36) - : : : : : +- CometFilter (35) - : : : : : +- CometScan parquet spark_catalog.default.customer (34) - : : : : +- BroadcastExchange (40) - : : : : +- * ColumnarToRow (39) - : : : : +- CometFilter (38) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (37) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (68) - : : +- * HashAggregate (67) - : : +- Exchange (66) - : : +- * HashAggregate (65) - : : +- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * ColumnarToRow (55) - : : : : +- CometFilter (54) - : : : : +- CometScan parquet spark_catalog.default.customer (53) - : : : +- BroadcastExchange (59) - : : : +- * ColumnarToRow (58) - : : : +- CometFilter (57) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (56) - : : +- ReusedExchange (62) - : +- BroadcastExchange (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (79) - : : +- * BroadcastHashJoin Inner BuildRight (78) - : : :- * ColumnarToRow (73) - : : : +- CometFilter (72) - : : : +- CometScan parquet spark_catalog.default.customer (71) - : : +- BroadcastExchange (77) - : : +- * ColumnarToRow (76) - : : +- CometFilter (75) - : : +- CometScan parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (80) - +- BroadcastExchange (105) - +- * HashAggregate (104) - +- Exchange (103) - +- * HashAggregate (102) - +- * Project (101) - +- * BroadcastHashJoin Inner BuildRight (100) - :- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * ColumnarToRow (92) - : : +- CometFilter (91) - : : +- CometScan parquet spark_catalog.default.customer (90) - : +- BroadcastExchange (96) - : +- * ColumnarToRow (95) - : +- CometFilter (94) - : +- CometScan parquet spark_catalog.default.web_sales (93) - +- ReusedExchange (99) +TakeOrderedAndProject (120) ++- * Project (119) + +- * BroadcastHashJoin Inner BuildRight (118) + :- * Project (99) + : +- * BroadcastHashJoin Inner BuildRight (98) + : :- * Project (78) + : : +- * BroadcastHashJoin Inner BuildRight (77) + : : :- * Project (58) + : : : +- * BroadcastHashJoin Inner BuildRight (57) + : : : :- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * ColumnarToRow (18) + : : : : : +- CometFilter (17) + : : : : : +- CometHashAggregate (16) + : : : : : +- CometColumnarExchange (15) + : : : : : +- RowToColumnar (14) + : : : : : +- * HashAggregate (13) + : : : : : +- * Project (12) + : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : : +- ReusedExchange (10) + : : : : +- BroadcastExchange (36) + : : : : +- * ColumnarToRow (35) + : : : : +- CometHashAggregate (34) + : : : : +- CometColumnarExchange (33) + : : : : +- RowToColumnar (32) + : : : : +- * HashAggregate (31) + : : : : +- * Project (30) + : : : : +- * BroadcastHashJoin Inner BuildRight (29) + : : : : :- * Project (27) + : : : : : +- * BroadcastHashJoin Inner BuildRight (26) + : : : : : :- * ColumnarToRow (21) + : : : : : : +- CometFilter (20) + : : : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : : : +- BroadcastExchange (25) + : : : : : +- * ColumnarToRow (24) + : : : : : +- CometFilter (23) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (22) + : : : : +- ReusedExchange (28) + : : : +- BroadcastExchange (56) + : : : +- * ColumnarToRow (55) + : : : +- CometFilter (54) + : : : +- CometHashAggregate (53) + : : : +- CometColumnarExchange (52) + : : : +- RowToColumnar (51) + : : : +- * HashAggregate (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : :- * Project (46) + : : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : : :- * ColumnarToRow (40) + : : : : : +- CometFilter (39) + : : : : : +- CometScan parquet spark_catalog.default.customer (38) + : : : : +- BroadcastExchange (44) + : : : : +- * ColumnarToRow (43) + : : : : +- CometFilter (42) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) + : : : +- ReusedExchange (47) + : : +- BroadcastExchange (76) + : : +- * ColumnarToRow (75) + : : +- CometHashAggregate (74) + : : +- CometColumnarExchange (73) + : : +- RowToColumnar (72) + : : +- * HashAggregate (71) + : : +- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * Project (67) + : : : +- * BroadcastHashJoin Inner BuildRight (66) + : : : :- * ColumnarToRow (61) + : : : : +- CometFilter (60) + : : : : +- CometScan parquet spark_catalog.default.customer (59) + : : : +- BroadcastExchange (65) + : : : +- * ColumnarToRow (64) + : : : +- CometFilter (63) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (62) + : : +- ReusedExchange (68) + : +- BroadcastExchange (97) + : +- * ColumnarToRow (96) + : +- CometFilter (95) + : +- CometHashAggregate (94) + : +- CometColumnarExchange (93) + : +- RowToColumnar (92) + : +- * HashAggregate (91) + : +- * Project (90) + : +- * BroadcastHashJoin Inner BuildRight (89) + : :- * Project (87) + : : +- * BroadcastHashJoin Inner BuildRight (86) + : : :- * ColumnarToRow (81) + : : : +- CometFilter (80) + : : : +- CometScan parquet spark_catalog.default.customer (79) + : : +- BroadcastExchange (85) + : : +- * ColumnarToRow (84) + : : +- CometFilter (83) + : : +- CometScan parquet spark_catalog.default.web_sales (82) + : +- ReusedExchange (88) + +- BroadcastExchange (117) + +- * ColumnarToRow (116) + +- CometHashAggregate (115) + +- CometColumnarExchange (114) + +- RowToColumnar (113) + +- * HashAggregate (112) + +- * Project (111) + +- * BroadcastHashJoin Inner BuildRight (110) + :- * Project (108) + : +- * BroadcastHashJoin Inner BuildRight (107) + : :- * ColumnarToRow (102) + : : +- CometFilter (101) + : : +- CometScan parquet spark_catalog.default.customer (100) + : +- BroadcastExchange (106) + : +- * ColumnarToRow (105) + : +- CometFilter (104) + : +- CometScan parquet spark_catalog.default.web_sales (103) + +- ReusedExchange (109) (unknown) Scan parquet spark_catalog.default.customer @@ -152,7 +164,7 @@ Join condition: None Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -(10) ReusedExchange [Reuses operator id: 112] +(10) ReusedExchange [Reuses operator id: 124] Output [2]: [d_date_sk#16, d_year#17] (11) BroadcastHashJoin [codegen id : 3] @@ -172,476 +184,500 @@ Functions [1]: [partial_sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] -(14) Exchange +(14) RowToColumnar Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 24] +(15) CometColumnarExchange +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] Functions [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))] -Aggregate Attributes [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22] -Results [2]: [c_customer_id#2 AS customer_id#23, sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22 AS year_total#24] -(16) Filter [codegen id : 24] -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.000000)) +(17) CometFilter +Input [2]: [customer_id#22, year_total#23] +Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.000000)) + +(18) ColumnarToRow [codegen id : 24] +Input [2]: [customer_id#22, year_total#23] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_customer_id#26)) +(20) CometFilter +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) -(19) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +(21) ColumnarToRow [codegen id : 6] +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] (unknown) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Output [6]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ss_sold_date_sk#37), dynamicpruningexpression(ss_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Condition : isnotnull(ss_customer_sk#33) +(23) CometFilter +Input [6]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] +Condition : isnotnull(ss_customer_sk#32) -(22) ColumnarToRow [codegen id : 4] -Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +(24) ColumnarToRow [codegen id : 4] +Input [6]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] -(23) BroadcastExchange -Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +(25) BroadcastExchange +Input [6]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#25] -Right keys [1]: [ss_customer_sk#33] +(26) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#24] +Right keys [1]: [ss_customer_sk#32] Join type: Inner Join condition: None -(25) Project [codegen id : 6] -Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Input [14]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +(27) Project [codegen id : 6] +Output [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] +Input [14]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] -(26) ReusedExchange [Reuses operator id: 116] -Output [2]: [d_date_sk#40, d_year#41] +(28) ReusedExchange [Reuses operator id: 128] +Output [2]: [d_date_sk#39, d_year#40] -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] +(29) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#37] +Right keys [1]: [d_date_sk#39] Join type: Inner Join condition: None -(28) Project [codegen id : 6] -Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] -Input [14]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] - -(29) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] -Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] -Functions [1]: [partial_sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] - -(30) Exchange -Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] -Arguments: hashpartitioning(c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] -Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] -Functions [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] -Aggregate Attributes [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22] -Results [8]: [c_customer_id#26 AS customer_id#46, c_first_name#27 AS customer_first_name#47, c_last_name#28 AS customer_last_name#48, c_preferred_cust_flag#29 AS customer_preferred_cust_flag#49, c_birth_country#30 AS customer_birth_country#50, c_login#31 AS customer_login#51, c_email_address#32 AS customer_email_address#52, sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22 AS year_total#53] - -(32) BroadcastExchange -Input [8]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53] +(30) Project [codegen id : 6] +Output [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, d_year#40] +Input [14]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37, d_date_sk#39, d_year#40] + +(31) HashAggregate [codegen id : 6] +Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, d_year#40] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40] +Functions [1]: [partial_sum(((((ss_ext_list_price#36 - ss_ext_wholesale_cost#35) - ss_ext_discount_amt#33) + ss_ext_sales_price#34) / 2))] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40, sum#43, isEmpty#44] + +(32) RowToColumnar +Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40, sum#43, isEmpty#44] + +(33) CometColumnarExchange +Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40, sum#43, isEmpty#44] +Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometHashAggregate +Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40, sum#43, isEmpty#44] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40] +Functions [1]: [sum(((((ss_ext_list_price#36 - ss_ext_wholesale_cost#35) - ss_ext_discount_amt#33) + ss_ext_sales_price#34) / 2))] + +(35) ColumnarToRow [codegen id : 7] +Input [8]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#52] + +(36) BroadcastExchange +Input [8]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#52] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#46] +(37) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [customer_id#22] +Right keys [1]: [customer_id#45] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] +Output [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter -Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] -Condition : (isnotnull(c_customer_sk#54) AND isnotnull(c_customer_id#55)) +(39) CometFilter +Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] +Condition : (isnotnull(c_customer_sk#53) AND isnotnull(c_customer_id#54)) -(36) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] +(40) ColumnarToRow [codegen id : 10] +Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Output [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#67), dynamicpruningexpression(cs_sold_date_sk#67 IN dynamicpruning#68)] +PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -Condition : isnotnull(cs_bill_customer_sk#62) +(42) CometFilter +Input [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] +Condition : isnotnull(cs_bill_customer_sk#61) -(39) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +(43) ColumnarToRow [codegen id : 8] +Input [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] -(40) BroadcastExchange -Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +(44) BroadcastExchange +Input [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#54] -Right keys [1]: [cs_bill_customer_sk#62] +(45) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#53] +Right keys [1]: [cs_bill_customer_sk#61] Join type: Inner Join condition: None -(42) Project [codegen id : 10] -Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -Input [14]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +(46) Project [codegen id : 10] +Output [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] +Input [14]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] -(43) ReusedExchange [Reuses operator id: 112] -Output [2]: [d_date_sk#69, d_year#70] +(47) ReusedExchange [Reuses operator id: 124] +Output [2]: [d_date_sk#68, d_year#69] -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#67] -Right keys [1]: [d_date_sk#69] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#66] +Right keys [1]: [d_date_sk#68] Join type: Inner Join condition: None -(45) Project [codegen id : 10] -Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] -Input [14]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67, d_date_sk#69, d_year#70] - -(46) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] -Functions [1]: [partial_sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] -Aggregate Attributes [2]: [sum#71, isEmpty#72] -Results [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] - -(47) Exchange -Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(48) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] -Functions [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75] -Results [2]: [c_customer_id#55 AS customer_id#76, sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75 AS year_total#77] - -(49) Filter [codegen id : 11] -Input [2]: [customer_id#76, year_total#77] -Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.000000)) - -(50) BroadcastExchange -Input [2]: [customer_id#76, year_total#77] +(49) Project [codegen id : 10] +Output [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, d_year#69] +Input [14]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66, d_date_sk#68, d_year#69] + +(50) HashAggregate [codegen id : 10] +Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, d_year#69] +Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69] +Functions [1]: [partial_sum(((((cs_ext_list_price#65 - cs_ext_wholesale_cost#64) - cs_ext_discount_amt#62) + cs_ext_sales_price#63) / 2))] +Aggregate Attributes [2]: [sum#70, isEmpty#71] +Results [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69, sum#72, isEmpty#73] + +(51) RowToColumnar +Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69, sum#72, isEmpty#73] + +(52) CometColumnarExchange +Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69, sum#72, isEmpty#73] +Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(53) CometHashAggregate +Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69, sum#72, isEmpty#73] +Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69] +Functions [1]: [sum(((((cs_ext_list_price#65 - cs_ext_wholesale_cost#64) - cs_ext_discount_amt#62) + cs_ext_sales_price#63) / 2))] + +(54) CometFilter +Input [2]: [customer_id#74, year_total#75] +Condition : (isnotnull(year_total#75) AND (year_total#75 > 0.000000)) + +(55) ColumnarToRow [codegen id : 11] +Input [2]: [customer_id#74, year_total#75] + +(56) BroadcastExchange +Input [2]: [customer_id#74, year_total#75] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#76] +(57) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [customer_id#22] +Right keys [1]: [customer_id#74] Join type: Inner Join condition: None -(52) Project [codegen id : 24] -Output [11]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77] -Input [12]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, customer_id#76, year_total#77] +(58) Project [codegen id : 24] +Output [11]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#52, year_total#75] +Input [12]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#52, customer_id#74, year_total#75] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] +Output [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] -Condition : (isnotnull(c_customer_sk#78) AND isnotnull(c_customer_id#79)) +(60) CometFilter +Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] +Condition : (isnotnull(c_customer_sk#76) AND isnotnull(c_customer_id#77)) -(55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] +(61) ColumnarToRow [codegen id : 14] +Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +Output [6]: [cs_bill_customer_sk#84, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(cs_sold_date_sk#89), dynamicpruningexpression(cs_sold_date_sk#89 IN dynamicpruning#90)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] -Condition : isnotnull(cs_bill_customer_sk#86) +(63) CometFilter +Input [6]: [cs_bill_customer_sk#84, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] +Condition : isnotnull(cs_bill_customer_sk#84) -(58) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +(64) ColumnarToRow [codegen id : 12] +Input [6]: [cs_bill_customer_sk#84, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] -(59) BroadcastExchange -Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +(65) BroadcastExchange +Input [6]: [cs_bill_customer_sk#84, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#78] -Right keys [1]: [cs_bill_customer_sk#86] +(66) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#76] +Right keys [1]: [cs_bill_customer_sk#84] Join type: Inner Join condition: None -(61) Project [codegen id : 14] -Output [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] -Input [14]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +(67) Project [codegen id : 14] +Output [12]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] +Input [14]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, cs_bill_customer_sk#84, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] -(62) ReusedExchange [Reuses operator id: 116] -Output [2]: [d_date_sk#93, d_year#94] +(68) ReusedExchange [Reuses operator id: 128] +Output [2]: [d_date_sk#91, d_year#92] -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#91] -Right keys [1]: [d_date_sk#93] +(69) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [cs_sold_date_sk#89] +Right keys [1]: [d_date_sk#91] Join type: Inner Join condition: None -(64) Project [codegen id : 14] -Output [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] -Input [14]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91, d_date_sk#93, d_year#94] - -(65) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] -Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94] -Functions [1]: [partial_sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))] -Aggregate Attributes [2]: [sum#95, isEmpty#96] -Results [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] - -(66) Exchange -Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] -Arguments: hashpartitioning(c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(67) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] -Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94] -Functions [1]: [sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))#75] -Results [2]: [c_customer_id#79 AS customer_id#99, sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))#75 AS year_total#100] - -(68) BroadcastExchange -Input [2]: [customer_id#99, year_total#100] +(70) Project [codegen id : 14] +Output [12]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, d_year#92] +Input [14]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89, d_date_sk#91, d_year#92] + +(71) HashAggregate [codegen id : 14] +Input [12]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, d_year#92] +Keys [8]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92] +Functions [1]: [partial_sum(((((cs_ext_list_price#88 - cs_ext_wholesale_cost#87) - cs_ext_discount_amt#85) + cs_ext_sales_price#86) / 2))] +Aggregate Attributes [2]: [sum#93, isEmpty#94] +Results [10]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92, sum#95, isEmpty#96] + +(72) RowToColumnar +Input [10]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92, sum#95, isEmpty#96] + +(73) CometColumnarExchange +Input [10]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92, sum#95, isEmpty#96] +Arguments: hashpartitioning(c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(74) CometHashAggregate +Input [10]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92, sum#95, isEmpty#96] +Keys [8]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92] +Functions [1]: [sum(((((cs_ext_list_price#88 - cs_ext_wholesale_cost#87) - cs_ext_discount_amt#85) + cs_ext_sales_price#86) / 2))] + +(75) ColumnarToRow [codegen id : 15] +Input [2]: [customer_id#97, year_total#98] + +(76) BroadcastExchange +Input [2]: [customer_id#97, year_total#98] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(69) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#99] +(77) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [customer_id#22] +Right keys [1]: [customer_id#97] Join type: Inner -Join condition: (CASE WHEN (year_total#77 > 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END) +Join condition: (CASE WHEN (year_total#75 > 0.000000) THEN (year_total#98 / year_total#75) END > CASE WHEN (year_total#23 > 0.000000) THEN (year_total#52 / year_total#23) END) -(70) Project [codegen id : 24] -Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100] -Input [13]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77, customer_id#99, year_total#100] +(78) Project [codegen id : 24] +Output [10]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#75, year_total#98] +Input [13]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#52, year_total#75, customer_id#97, year_total#98] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Output [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(72) CometFilter -Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] -Condition : (isnotnull(c_customer_sk#101) AND isnotnull(c_customer_id#102)) +(80) CometFilter +Input [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] +Condition : (isnotnull(c_customer_sk#99) AND isnotnull(c_customer_id#100)) -(73) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +(81) ColumnarToRow [codegen id : 18] +Input [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] (unknown) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +Output [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#114), dynamicpruningexpression(ws_sold_date_sk#114 IN dynamicpruning#115)] +PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_sold_date_sk#112 IN dynamicpruning#113)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(75) CometFilter -Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] -Condition : isnotnull(ws_bill_customer_sk#109) +(83) CometFilter +Input [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] +Condition : isnotnull(ws_bill_customer_sk#107) -(76) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +(84) ColumnarToRow [codegen id : 16] +Input [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] -(77) BroadcastExchange -Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +(85) BroadcastExchange +Input [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#101] -Right keys [1]: [ws_bill_customer_sk#109] +(86) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_customer_sk#99] +Right keys [1]: [ws_bill_customer_sk#107] Join type: Inner Join condition: None -(79) Project [codegen id : 18] -Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] -Input [14]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +(87) Project [codegen id : 18] +Output [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] +Input [14]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] -(80) ReusedExchange [Reuses operator id: 112] -Output [2]: [d_date_sk#116, d_year#117] +(88) ReusedExchange [Reuses operator id: 124] +Output [2]: [d_date_sk#114, d_year#115] -(81) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#114] -Right keys [1]: [d_date_sk#116] +(89) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_sold_date_sk#112] +Right keys [1]: [d_date_sk#114] Join type: Inner Join condition: None -(82) Project [codegen id : 18] -Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] -Input [14]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114, d_date_sk#116, d_year#117] - -(83) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] -Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] -Functions [1]: [partial_sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] -Aggregate Attributes [2]: [sum#118, isEmpty#119] -Results [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] - -(84) Exchange -Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] -Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(85) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] -Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] -Functions [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122] -Results [2]: [c_customer_id#102 AS customer_id#123, sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122 AS year_total#124] - -(86) Filter [codegen id : 19] -Input [2]: [customer_id#123, year_total#124] -Condition : (isnotnull(year_total#124) AND (year_total#124 > 0.000000)) - -(87) BroadcastExchange -Input [2]: [customer_id#123, year_total#124] +(90) Project [codegen id : 18] +Output [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, d_year#115] +Input [14]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112, d_date_sk#114, d_year#115] + +(91) HashAggregate [codegen id : 18] +Input [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, d_year#115] +Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115] +Functions [1]: [partial_sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))] +Aggregate Attributes [2]: [sum#116, isEmpty#117] +Results [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115, sum#118, isEmpty#119] + +(92) RowToColumnar +Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115, sum#118, isEmpty#119] + +(93) CometColumnarExchange +Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115, sum#118, isEmpty#119] +Arguments: hashpartitioning(c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(94) CometHashAggregate +Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115, sum#118, isEmpty#119] +Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115] +Functions [1]: [sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))] + +(95) CometFilter +Input [2]: [customer_id#120, year_total#121] +Condition : (isnotnull(year_total#121) AND (year_total#121 > 0.000000)) + +(96) ColumnarToRow [codegen id : 19] +Input [2]: [customer_id#120, year_total#121] + +(97) BroadcastExchange +Input [2]: [customer_id#120, year_total#121] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] -(88) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#123] +(98) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [customer_id#22] +Right keys [1]: [customer_id#120] Join type: Inner Join condition: None -(89) Project [codegen id : 24] -Output [11]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124] -Input [12]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, customer_id#123, year_total#124] +(99) Project [codegen id : 24] +Output [11]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#75, year_total#98, year_total#121] +Input [12]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#75, year_total#98, customer_id#120, year_total#121] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +Output [8]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(91) CometFilter -Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] -Condition : (isnotnull(c_customer_sk#125) AND isnotnull(c_customer_id#126)) +(101) CometFilter +Input [8]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129] +Condition : (isnotnull(c_customer_sk#122) AND isnotnull(c_customer_id#123)) -(92) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +(102) ColumnarToRow [codegen id : 22] +Input [8]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129] (unknown) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +Output [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#138), dynamicpruningexpression(ws_sold_date_sk#138 IN dynamicpruning#139)] +PartitionFilters: [isnotnull(ws_sold_date_sk#135), dynamicpruningexpression(ws_sold_date_sk#135 IN dynamicpruning#136)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(94) CometFilter -Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] -Condition : isnotnull(ws_bill_customer_sk#133) +(104) CometFilter +Input [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] +Condition : isnotnull(ws_bill_customer_sk#130) -(95) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +(105) ColumnarToRow [codegen id : 20] +Input [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] -(96) BroadcastExchange -Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +(106) BroadcastExchange +Input [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -(97) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#125] -Right keys [1]: [ws_bill_customer_sk#133] +(107) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [c_customer_sk#122] +Right keys [1]: [ws_bill_customer_sk#130] Join type: Inner Join condition: None -(98) Project [codegen id : 22] -Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] -Input [14]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +(108) Project [codegen id : 22] +Output [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] +Input [14]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] -(99) ReusedExchange [Reuses operator id: 116] -Output [2]: [d_date_sk#140, d_year#141] +(109) ReusedExchange [Reuses operator id: 128] +Output [2]: [d_date_sk#137, d_year#138] -(100) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#138] -Right keys [1]: [d_date_sk#140] +(110) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_sold_date_sk#135] +Right keys [1]: [d_date_sk#137] Join type: Inner Join condition: None -(101) Project [codegen id : 22] -Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] -Input [14]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138, d_date_sk#140, d_year#141] - -(102) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] -Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] -Functions [1]: [partial_sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] -Aggregate Attributes [2]: [sum#142, isEmpty#143] -Results [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] - -(103) Exchange -Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] -Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(104) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] -Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] -Functions [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122] -Results [2]: [c_customer_id#126 AS customer_id#146, sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122 AS year_total#147] - -(105) BroadcastExchange -Input [2]: [customer_id#146, year_total#147] +(111) Project [codegen id : 22] +Output [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, d_year#138] +Input [14]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135, d_date_sk#137, d_year#138] + +(112) HashAggregate [codegen id : 22] +Input [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, d_year#138] +Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138] +Functions [1]: [partial_sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))] +Aggregate Attributes [2]: [sum#139, isEmpty#140] +Results [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138, sum#141, isEmpty#142] + +(113) RowToColumnar +Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138, sum#141, isEmpty#142] + +(114) CometColumnarExchange +Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138, sum#141, isEmpty#142] +Arguments: hashpartitioning(c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] + +(115) CometHashAggregate +Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138, sum#141, isEmpty#142] +Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138] +Functions [1]: [sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))] + +(116) ColumnarToRow [codegen id : 23] +Input [2]: [customer_id#143, year_total#144] + +(117) BroadcastExchange +Input [2]: [customer_id#143, year_total#144] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] -(106) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#146] +(118) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [customer_id#22] +Right keys [1]: [customer_id#143] Join type: Inner -Join condition: (CASE WHEN (year_total#77 > 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#124 > 0.000000) THEN (year_total#147 / year_total#124) END) +Join condition: (CASE WHEN (year_total#75 > 0.000000) THEN (year_total#98 / year_total#75) END > CASE WHEN (year_total#121 > 0.000000) THEN (year_total#144 / year_total#121) END) -(107) Project [codegen id : 24] -Output [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] -Input [13]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124, customer_id#146, year_total#147] +(119) Project [codegen id : 24] +Output [7]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51] +Input [13]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#75, year_total#98, year_total#121, customer_id#143, year_total#144] -(108) TakeOrderedAndProject -Input [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] -Arguments: 100, [customer_id#46 ASC NULLS FIRST, customer_first_name#47 ASC NULLS FIRST, customer_last_name#48 ASC NULLS FIRST, customer_preferred_cust_flag#49 ASC NULLS FIRST, customer_birth_country#50 ASC NULLS FIRST, customer_login#51 ASC NULLS FIRST, customer_email_address#52 ASC NULLS FIRST], [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] +(120) TakeOrderedAndProject +Input [7]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51] +Arguments: 100, [customer_id#45 ASC NULLS FIRST, customer_first_name#46 ASC NULLS FIRST, customer_last_name#47 ASC NULLS FIRST, customer_preferred_cust_flag#48 ASC NULLS FIRST, customer_birth_country#49 ASC NULLS FIRST, customer_login#50 ASC NULLS FIRST, customer_email_address#51 ASC NULLS FIRST], [customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 -BroadcastExchange (112) -+- * ColumnarToRow (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +BroadcastExchange (124) ++- * ColumnarToRow (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) (unknown) Scan parquet spark_catalog.default.date_dim @@ -651,48 +687,48 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(110) CometFilter +(122) CometFilter Input [2]: [d_date_sk#16, d_year#17] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(111) ColumnarToRow [codegen id : 1] +(123) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] -(112) BroadcastExchange +(124) BroadcastExchange Input [2]: [d_date_sk#16, d_year#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 -BroadcastExchange (116) -+- * ColumnarToRow (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#37 IN dynamicpruning#38 +BroadcastExchange (128) ++- * ColumnarToRow (127) + +- CometFilter (126) + +- CometScan parquet spark_catalog.default.date_dim (125) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_year#41] +Output [2]: [d_date_sk#39, d_year#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter -Input [2]: [d_date_sk#40, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) +(126) CometFilter +Input [2]: [d_date_sk#39, d_year#40] +Condition : ((isnotnull(d_year#40) AND (d_year#40 = 2002)) AND isnotnull(d_date_sk#39)) -(115) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#40, d_year#41] +(127) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#39, d_year#40] -(116) BroadcastExchange -Input [2]: [d_date_sk#40, d_year#41] +(128) BroadcastExchange +Input [2]: [d_date_sk#39, d_year#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] -Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#39 +Subquery:4 Hosting operator id = 62 Hosting Expression = cs_sold_date_sk#89 IN dynamicpruning#38 -Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#114 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#39 +Subquery:6 Hosting operator id = 103 Hosting Expression = ws_sold_date_sk#135 IN dynamicpruning#38 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 99e255a0e4..3299726c59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -9,43 +9,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 - WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [year_total] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 + RowToColumnar + WholeStageCodegen (3) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -56,124 +26,166 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + ColumnarToRow + InputAdapter + CometFilter [year_total] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + ColumnarToRow InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #14 WholeStageCodegen (19) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 - WholeStageCodegen (18) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + ColumnarToRow + InputAdapter + CometFilter [year_total] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 + RowToColumnar + WholeStageCodegen (18) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #17 WholeStageCodegen (23) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + ColumnarToRow InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 - WholeStageCodegen (22) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 + RowToColumnar + WholeStageCodegen (22) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #19 + WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #19 - WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt index 9f6036a23d..89bb2af3e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt @@ -1,36 +1,38 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * ColumnarToRow (12) - : : : +- CometProject (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (5) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.warehouse (13) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- ReusedExchange (26) +* ColumnarToRow (34) ++- CometTakeOrderedAndProject (33) + +- CometHashAggregate (32) + +- CometColumnarExchange (31) + +- RowToColumnar (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * ColumnarToRow (12) + : : : +- CometProject (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (5) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.warehouse (13) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometProject (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- ReusedExchange (26) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -45,9 +47,9 @@ ReadSchema: struct= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] -Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27] -Results [4]: [w_state#12, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26 AS sales_before#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27 AS sales_after#29] -(32) TakeOrderedAndProject -Input [4]: [w_state#12, i_item_id#14, sales_before#28, sales_after#29] -Arguments: 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#28, sales_after#29] +(33) CometTakeOrderedAndProject +Input [4]: [w_state#12, i_item_id#14, sales_before#26, sales_after#27] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#12 ASC NULLS FIRST,i_item_id#14 ASC NULLS FIRST], output=[w_state#12,i_item_id#14,sales_before#26,sales_after#27]), 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#26, sales_after#27] + +(34) ColumnarToRow [codegen id : 5] +Input [4]: [w_state#12, i_item_id#14, sales_before#26, sales_after#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.date_dim (35) (unknown) Scan parquet spark_catalog.default.date_dim @@ -199,14 +205,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(36) CometFilter Input [2]: [d_date_sk#16, d_date#17] Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) -(35) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] -(36) BroadcastExchange +(38) BroadcastExchange Input [2]: [d_date_sk#16, d_date#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index 98d39c58d5..541de7ec5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -1,50 +1,52 @@ -TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - WholeStageCodegen (5) - HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_state,i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometExchange [cs_order_number,cs_item_sk] #2 - CometFilter [cs_warehouse_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_order_number,cr_item_sk] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] +WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] + CometHashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [w_state,i_item_id] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] + CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometSort [cs_order_number,cs_item_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #2 + CometFilter [cs_warehouse_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [cr_order_number,cr_item_sk] + CometColumnarExchange [cr_order_number,cr_item_sk] #4 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) + BroadcastExchange #6 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] + CometProject [i_item_sk,i_item_id] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 2308990771..ee6d6a04ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * BroadcastHashJoin Inner BuildRight (15) - :- * ColumnarToRow (4) - : +- CometProject (3) - : +- CometFilter (2) - : +- CometScan parquet spark_catalog.default.item (1) - +- BroadcastExchange (14) - +- * ColumnarToRow (13) - +- CometProject (12) - +- CometFilter (11) - +- CometHashAggregate (10) - +- CometExchange (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan parquet spark_catalog.default.item (5) +* ColumnarToRow (22) ++- CometTakeOrderedAndProject (21) + +- CometHashAggregate (20) + +- CometColumnarExchange (19) + +- RowToColumnar (18) + +- * HashAggregate (17) + +- * Project (16) + +- * BroadcastHashJoin Inner BuildRight (15) + :- * ColumnarToRow (4) + : +- CometProject (3) + : +- CometFilter (2) + : +- CometScan parquet spark_catalog.default.item (1) + +- BroadcastExchange (14) + +- * ColumnarToRow (13) + +- CometProject (12) + +- CometFilter (11) + +- CometHashAggregate (10) + +- CometColumnarExchange (9) + +- CometHashAggregate (8) + +- CometProject (7) + +- CometFilter (6) + +- CometScan parquet spark_catalog.default.item (5) (unknown) Scan parquet spark_catalog.default.item @@ -59,9 +61,9 @@ Input [1]: [i_manufact#5] Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -(9) CometExchange +(9) CometColumnarExchange Input [2]: [i_manufact#5, count#9] -Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] (10) CometHashAggregate Input [2]: [i_manufact#5, count#9] @@ -100,18 +102,22 @@ Functions: [] Aggregate Attributes: [] Results [1]: [i_product_name#3] -(18) Exchange +(18) RowToColumnar Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(19) HashAggregate [codegen id : 3] +(19) CometColumnarExchange +Input [1]: [i_product_name#3] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(20) CometHashAggregate Input [1]: [i_product_name#3] Keys [1]: [i_product_name#3] Functions: [] -Aggregate Attributes: [] -Results [1]: [i_product_name#3] -(20) TakeOrderedAndProject +(21) CometTakeOrderedAndProject +Input [1]: [i_product_name#3] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#3 ASC NULLS FIRST], output=[i_product_name#3]), 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] + +(22) ColumnarToRow [codegen id : 3] Input [1]: [i_product_name#3] -Arguments: 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index 6c8ffadfdb..3c68188747 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -1,27 +1,29 @@ -TakeOrderedAndProject [i_product_name] - WholeStageCodegen (3) - HashAggregate [i_product_name] - InputAdapter - Exchange [i_product_name] #1 - WholeStageCodegen (2) - HashAggregate [i_product_name] - Project [i_product_name] - BroadcastHashJoin [i_manufact,i_manufact] - ColumnarToRow - InputAdapter - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] - CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_manufact] - CometFilter [item_cnt] - CometHashAggregate [i_manufact,count] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact] - CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] - CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] +WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometColumnarExchange [i_product_name] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [i_product_name] + Project [i_product_name] + BroadcastHashJoin [i_manufact,i_manufact] + ColumnarToRow + InputAdapter + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact] + CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_manufact] + CometFilter [item_cnt] + CometHashAggregate [i_manufact,count] + CometColumnarExchange [i_manufact] #3 + CometHashAggregate [i_manufact] + CometProject [i_manufact] + CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt index 958f358b7b..647bf61f60 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt @@ -1,25 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +* ColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometHashAggregate (21) + +- CometColumnarExchange (20) + +- RowToColumnar (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) (unknown) Scan parquet spark_catalog.default.date_dim @@ -108,18 +110,22 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] -(19) Exchange +(19) RowToColumnar Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) HashAggregate [codegen id : 4] +(20) CometColumnarExchange +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] +Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(21) CometHashAggregate Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] Keys [3]: [d_year#2, i_category_id#8, i_category#9] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_category_id#8, i_category#9, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum(ss_ext_sales_price)#14] -(21) TakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] -Arguments: 100, [sum(ss_ext_sales_price)#14 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] +(22) CometTakeOrderedAndProject +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#13] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#9 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#9,sum(ss_ext_sales_price)#13]), 100, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#13] + +(23) ColumnarToRow [codegen id : 4] +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt index 67906b8c7a..d33229197b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt @@ -1,31 +1,33 @@ -TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] - WholeStageCodegen (4) - HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] - InputAdapter - Exchange [d_year,i_category_id,i_category] #1 - WholeStageCodegen (3) - HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] - Project [d_year,ss_ext_sales_price,i_category_id,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] + CometHashAggregate [d_year,i_category_id,i_category,sum] + CometColumnarExchange [d_year,i_category_id,i_category] #1 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_category_id,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] + CometProject [i_item_sk,i_category_id,i_category] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt index 68b4b54005..4e1468e50c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt @@ -1,25 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.store (11) +* ColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometHashAggregate (21) + +- CometColumnarExchange (20) + +- RowToColumnar (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.store (11) (unknown) Scan parquet spark_catalog.default.date_dim @@ -108,18 +110,22 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) T Aggregate Attributes [7]: [sum#11, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17] Results [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] -(19) Exchange +(19) RowToColumnar Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(s_store_name#9, s_store_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) HashAggregate [codegen id : 4] +(20) CometColumnarExchange +Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(s_store_name#9, s_store_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(21) CometHashAggregate Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [2]: [s_store_name#9, s_store_id#8] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31] -Results [9]: [s_store_name#9, s_store_id#8, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25,17,2) AS sun_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26,17,2) AS mon_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27,17,2) AS tue_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28,17,2) AS wed_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29,17,2) AS thu_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30,17,2) AS fri_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31,17,2) AS sat_sales#38] -(21) TakeOrderedAndProject -Input [9]: [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] -Arguments: 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#32 ASC NULLS FIRST, mon_sales#33 ASC NULLS FIRST, tue_sales#34 ASC NULLS FIRST, wed_sales#35 ASC NULLS FIRST, thu_sales#36 ASC NULLS FIRST, fri_sales#37 ASC NULLS FIRST, sat_sales#38 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] +(22) CometTakeOrderedAndProject +Input [9]: [s_store_name#9, s_store_id#8, sun_sales#25, mon_sales#26, tue_sales#27, wed_sales#28, thu_sales#29, fri_sales#30, sat_sales#31] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST,s_store_id#8 ASC NULLS FIRST,sun_sales#25 ASC NULLS FIRST,mon_sales#26 ASC NULLS FIRST,tue_sales#27 ASC NULLS FIRST,wed_sales#28 ASC NULLS FIRST,thu_sales#29 ASC NULLS FIRST,fri_sales#30 ASC NULLS FIRST,sat_sales#31 ASC NULLS FIRST], output=[s_store_name#9,s_store_id#8,sun_sales#25,mon_sales#26,tue_sales#27,wed_sales#28,thu_sales#29,fri_sales#30,sat_sales#31]), 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#25 ASC NULLS FIRST, mon_sales#26 ASC NULLS FIRST, tue_sales#27 ASC NULLS FIRST, wed_sales#28 ASC NULLS FIRST, thu_sales#29 ASC NULLS FIRST, fri_sales#30 ASC NULLS FIRST, sat_sales#31 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#25, mon_sales#26, tue_sales#27, wed_sales#28, thu_sales#29, fri_sales#30, sat_sales#31] + +(23) ColumnarToRow [codegen id : 4] +Input [9]: [s_store_name#9, s_store_id#8, sun_sales#25, mon_sales#26, tue_sales#27, wed_sales#28, thu_sales#29, fri_sales#30, sat_sales#31] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt index ef20430969..0980f4764e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt @@ -1,31 +1,33 @@ -TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - WholeStageCodegen (4) - HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - InputAdapter - Exchange [s_store_name,s_store_id] #1 - WholeStageCodegen (3) - HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,ss_sales_price,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [d_day_name,ss_store_sk,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] +WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] + CometColumnarExchange [s_store_name,s_store_id] #1 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [d_day_name,ss_sales_price,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [d_day_name,ss_store_sk,ss_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_day_name] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + CometProject [s_store_sk,s_store_id,s_store_name] + CometFilter [s_gmt_offset,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index d7f13b274c..e3a07a650d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -12,10 +12,10 @@ TakeOrderedAndProject (33) : : : +- Window (11) : : : +- * ColumnarToRow (10) : : : +- CometSort (9) - : : : +- CometExchange (8) + : : : +- CometColumnarExchange (8) : : : +- CometFilter (7) : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) + : : : +- CometColumnarExchange (5) : : : +- CometHashAggregate (4) : : : +- CometProject (3) : : : +- CometFilter (2) @@ -54,9 +54,9 @@ Input [2]: [ss_item_sk#1, ss_net_profit#3] Keys [1]: [ss_item_sk#1] Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -(5) CometExchange +(5) CometColumnarExchange Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] (6) CometHashAggregate Input [3]: [ss_item_sk#1, sum#5, count#6] @@ -67,9 +67,9 @@ Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] Input [2]: [item_sk#7, rank_col#8] Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) -(8) CometExchange +(8) CometColumnarExchange Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (9) CometSort Input [2]: [item_sk#7, rank_col#8] @@ -180,7 +180,7 @@ Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#19, worst_per Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] * ColumnarToRow (40) +- CometHashAggregate (39) - +- CometExchange (38) + +- CometColumnarExchange (38) +- CometHashAggregate (37) +- CometProject (36) +- CometFilter (35) @@ -207,9 +207,9 @@ Input [2]: [ss_store_sk#22, ss_net_profit#23] Keys [1]: [ss_store_sk#22] Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#23))] -(38) CometExchange +(38) CometColumnarExchange Input [3]: [ss_store_sk#22, sum#25, count#26] -Arguments: hashpartitioning(ss_store_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(ss_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] (39) CometHashAggregate Input [3]: [ss_store_sk#22, sum#25, count#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index bc065345c8..faccc1ec5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -17,20 +17,20 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] ColumnarToRow InputAdapter CometSort [rank_col] - CometExchange #1 + CometColumnarExchange #1 CometFilter [rank_col] Subquery #1 WholeStageCodegen (1) ColumnarToRow InputAdapter CometHashAggregate [ss_store_sk,sum,count] - CometExchange [ss_store_sk] #3 + CometColumnarExchange [ss_store_sk] #3 CometHashAggregate [ss_store_sk,ss_net_profit] CometProject [ss_store_sk,ss_net_profit] CometFilter [ss_store_sk,ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometHashAggregate [ss_item_sk,sum,count] - CometExchange [ss_item_sk] #2 + CometColumnarExchange [ss_item_sk] #2 CometHashAggregate [ss_item_sk,ss_net_profit] CometProject [ss_item_sk,ss_net_profit] CometFilter [ss_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index d5d860a393..ac8a1f4f7a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -1,40 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * Filter (31) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.customer (4) - : : : +- BroadcastExchange (13) - : : : +- * ColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (16) - : +- BroadcastExchange (22) - : +- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.item (25) +* ColumnarToRow (38) ++- CometTakeOrderedAndProject (37) + +- CometHashAggregate (36) + +- CometColumnarExchange (35) + +- RowToColumnar (34) + +- * HashAggregate (33) + +- * Project (32) + +- * Filter (31) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.customer (4) + : : : +- BroadcastExchange (13) + : : : +- * ColumnarToRow (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.customer_address (10) + : : +- ReusedExchange (16) + : +- BroadcastExchange (22) + : +- * ColumnarToRow (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.item (25) (unknown) Scan parquet spark_catalog.default.web_sales @@ -108,7 +110,7 @@ Join condition: None Output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11] Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#11] -(16) ReusedExchange [Reuses operator id: 41] +(16) ReusedExchange [Reuses operator id: 43] Output [1]: [d_date_sk#12] (17) BroadcastHashJoin [codegen id : 6] @@ -192,50 +194,54 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] Aggregate Attributes [1]: [sum#17] Results [3]: [ca_zip#11, ca_city#10, sum#18] -(34) Exchange +(34) RowToColumnar Input [3]: [ca_zip#11, ca_city#10, sum#18] -Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(35) HashAggregate [codegen id : 7] +(35) CometColumnarExchange +Input [3]: [ca_zip#11, ca_city#10, sum#18] +Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(36) CometHashAggregate Input [3]: [ca_zip#11, ca_city#10, sum#18] Keys [2]: [ca_zip#11, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#19] -Results [3]: [ca_zip#11, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#19,17,2) AS sum(ws_sales_price)#20] -(36) TakeOrderedAndProject -Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] -Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] +(37) CometTakeOrderedAndProject +Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#19] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#11 ASC NULLS FIRST,ca_city#10 ASC NULLS FIRST], output=[ca_zip#11,ca_city#10,sum(ws_sales_price)#19]), 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#11, ca_city#10, sum(ws_sales_price)#19] + +(38) ColumnarToRow [codegen id : 7] +Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (41) -+- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#21, d_qoy#22] +Output [3]: [d_date_sk#12, d_year#20, d_qoy#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(38) CometFilter -Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] -Condition : ((((isnotnull(d_qoy#22) AND isnotnull(d_year#21)) AND (d_qoy#22 = 2)) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#12)) +(40) CometFilter +Input [3]: [d_date_sk#12, d_year#20, d_qoy#21] +Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 2)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#12)) -(39) CometProject -Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] +(41) CometProject +Input [3]: [d_date_sk#12, d_year#20, d_qoy#21] Arguments: [d_date_sk#12], [d_date_sk#12] -(40) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#12] -(41) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index 383cbb7e3b..b4965dae7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -1,61 +1,63 @@ -TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (7) - HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - InputAdapter - Exchange [ca_zip,ca_city] #1 - WholeStageCodegen (6) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - Project [ws_sales_price,ca_city,ca_zip,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] +WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] + CometHashAggregate [ca_zip,ca_city,sum] + CometColumnarExchange [ca_zip,ca_city] #1 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] + Project [ws_sales_price,ca_city,ca_zip] + Filter [ca_zip,exists] + BroadcastHashJoin [i_item_id,i_item_id] + Project [ws_sales_price,ca_city,ca_zip,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #6 + WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [i_item_id] - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometProject [i_item_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index 84d65306e0..a5c06c49ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -1,43 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * HashAggregate (29) - : : +- Exchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (33) - : +- * ColumnarToRow (32) - : +- CometFilter (31) - : +- CometScan parquet spark_catalog.default.customer (30) - +- ReusedExchange (36) +TakeOrderedAndProject (41) ++- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * ColumnarToRow (31) + : : +- CometHashAggregate (30) + : : +- CometColumnarExchange (29) + : : +- RowToColumnar (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * Project (13) + : : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : : :- * Project (6) + : : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- ReusedExchange (4) + : : : : +- BroadcastExchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store (7) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.household_demographics (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.customer_address (21) + : +- BroadcastExchange (35) + : +- * ColumnarToRow (34) + : +- CometFilter (33) + : +- CometScan parquet spark_catalog.default.customer (32) + +- ReusedExchange (38) (unknown) Scan parquet spark_catalog.default.store_sales @@ -55,7 +57,7 @@ Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotn (3) ColumnarToRow [codegen id : 5] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -(4) ReusedExchange [Reuses operator id: 44] +(4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#10] (5) BroadcastHashJoin [codegen id : 5] @@ -167,91 +169,95 @@ Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(Unscale Aggregate Attributes [2]: [sum#18, sum#19] Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -(28) Exchange +(28) RowToColumnar Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) HashAggregate [codegen id : 8] +(29) CometColumnarExchange +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometHashAggregate Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#22, sum(UnscaledValue(ss_net_profit#7))#23] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#17 AS bought_city#24, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#22,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#23,17,2) AS profit#26] + +(31) ColumnarToRow [codegen id : 8] +Input [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#22, amt#23, profit#24] (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +Output [4]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) +(33) CometFilter +Input [4]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28] +Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#26)) -(32) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +(34) ColumnarToRow [codegen id : 6] +Input [4]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28] -(33) BroadcastExchange -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +(35) BroadcastExchange +Input [4]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#27] +Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(35) Project [codegen id : 8] -Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +(37) Project [codegen id : 8] +Output [7]: [ss_ticket_number#5, bought_city#22, amt#23, profit#24, c_current_addr_sk#26, c_first_name#27, c_last_name#28] +Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#22, amt#23, profit#24, c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28] -(36) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#31, ca_city#32] +(38) ReusedExchange [Reuses operator id: 24] +Output [2]: [ca_address_sk#29, ca_city#30] -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#28] -Right keys [1]: [ca_address_sk#31] +(39) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [c_current_addr_sk#26] +Right keys [1]: [ca_address_sk#29] Join type: Inner -Join condition: NOT (ca_city#32 = bought_city#24) +Join condition: NOT (ca_city#30 = bought_city#22) -(38) Project [codegen id : 8] -Output [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_city#32] +(40) Project [codegen id : 8] +Output [7]: [c_last_name#28, c_first_name#27, ca_city#30, bought_city#22, ss_ticket_number#5, amt#23, profit#24] +Input [9]: [ss_ticket_number#5, bought_city#22, amt#23, profit#24, c_current_addr_sk#26, c_first_name#27, c_last_name#28, ca_address_sk#29, ca_city#30] -(39) TakeOrderedAndProject -Input [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Arguments: 100, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#24 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] +(41) TakeOrderedAndProject +Input [7]: [c_last_name#28, c_first_name#27, ca_city#30, bought_city#22, ss_ticket_number#5, amt#23, profit#24] +Arguments: 100, [c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, ca_city#30 ASC NULLS FIRST, bought_city#22 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#28, c_first_name#27, ca_city#30, bought_city#22, ss_ticket_number#5, amt#23, profit#24] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#33, d_dow#34] +Output [3]: [d_date_sk#10, d_year#31, d_dow#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [3]: [d_date_sk#10, d_year#33, d_dow#34] -Condition : ((d_dow#34 IN (6,0) AND d_year#33 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) +(43) CometFilter +Input [3]: [d_date_sk#10, d_year#31, d_dow#32] +Condition : ((d_dow#32 IN (6,0) AND d_year#31 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(42) CometProject -Input [3]: [d_date_sk#10, d_year#33, d_dow#34] +(44) CometProject +Input [3]: [d_date_sk#10, d_year#31, d_dow#32] Arguments: [d_date_sk#10], [d_date_sk#10] -(43) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(44) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index 04c59a2d35..4b9b7960e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -4,56 +4,58 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] + ColumnarToRow InputAdapter - Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dow,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_city,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index 9ea57de066..d7dfc7147d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * Filter (25) - : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) - +- BroadcastExchange (42) - +- * Project (41) - +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) +TakeOrderedAndProject (47) ++- * Project (46) + +- * BroadcastHashJoin Inner BuildRight (45) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * ColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- CometHashAggregate (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- BroadcastExchange (36) + : +- * Project (35) + : +- Window (34) + : +- * ColumnarToRow (33) + : +- CometSort (32) + : +- ReusedExchange (31) + +- BroadcastExchange (44) + +- * Project (43) + +- Window (42) + +- * ColumnarToRow (41) + +- CometSort (40) + +- ReusedExchange (39) (unknown) Scan parquet spark_catalog.default.item @@ -89,7 +91,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(10) ReusedExchange [Reuses operator id: 51] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -137,125 +139,124 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [1]: [sum#15] Results [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -(20) Exchange +(20) RowToColumnar Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) HashAggregate [codegen id : 5] +(21) CometColumnarExchange +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) CometHashAggregate Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#17] -Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS _w0#19] -(22) Exchange -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(23) CometColumnarExchange +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(23) Sort [codegen id : 6] -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 +(24) CometSort +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(24) Window -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(25) ColumnarToRow [codegen id : 5] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] + +(26) Window +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] +(27) Filter [codegen id : 6] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(26) Window -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] - -(27) Filter [codegen id : 22] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(28) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(29) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(30) HashAggregate [codegen id : 12] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] -Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] - -(31) Exchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(32) Sort [codegen id : 13] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 - -(33) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(34) Project [codegen id : 14] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] - -(35) BroadcastExchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] - -(36) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] +(28) Window +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] + +(29) Filter [codegen id : 19] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) + +(30) Project [codegen id : 19] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] + +(31) ReusedExchange [Reuses operator id: 23] +Output [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] + +(32) CometSort +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, s_store_name#23 ASC NULLS FIRST, s_company_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] + +(33) ColumnarToRow [codegen id : 11] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] + +(34) Window +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#27], [i_category#21, i_brand#22, s_store_name#23, s_company_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] + +(35) Project [codegen id : 12] +Output [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#17 AS sum_sales#28, rn#27] +Input [8]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17, rn#27] + +(36) BroadcastExchange +Input [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#28, rn#27] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 19] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] +Right keys [5]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, (rn#27 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +(38) Project [codegen id : 19] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#28, rn#27] + +(39) ReusedExchange [Reuses operator id: 23] +Output [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] -(38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +(40) CometSort +Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] +Arguments: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17], [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, s_store_name#31 ASC NULLS FIRST, s_company_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(39) Sort [codegen id : 20] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 +(41) ColumnarToRow [codegen id : 17] +Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] -(40) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +(42) Window +Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#29, i_brand#30, s_store_name#31, s_company_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(41) Project [codegen id : 21] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] +(43) Project [codegen id : 18] +Output [6]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#17 AS sum_sales#36, rn#35] +Input [8]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] -(42) BroadcastExchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] +(44) BroadcastExchange +Input [6]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#36, rn#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] +(45) BroadcastHashJoin [codegen id : 19] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] +Right keys [5]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +(46) Project [codegen id : 19] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#36 AS nsum#38] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#36, rn#35] -(45) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +(47) TakeOrderedAndProject +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim @@ -265,15 +266,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(49) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(51) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index 80b8da7b14..b93090ae54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (19) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,18 +8,18 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (6) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + RowToColumnar WholeStageCodegen (4) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] @@ -57,25 +57,23 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (11) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #7 + WholeStageCodegen (18) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (17) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt index a13a86a914..4c4c294385 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt @@ -1,32 +1,34 @@ == Physical Plan == -* HashAggregate (28) -+- Exchange (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.customer_demographics (10) - : +- BroadcastExchange (20) - : +- * ColumnarToRow (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.customer_address (16) - +- ReusedExchange (23) +* ColumnarToRow (30) ++- CometHashAggregate (29) + +- CometColumnarExchange (28) + +- RowToColumnar (27) + +- * HashAggregate (26) + +- * Project (25) + +- * BroadcastHashJoin Inner BuildRight (24) + :- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.customer_demographics (10) + : +- BroadcastExchange (20) + : +- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.customer_address (16) + +- ReusedExchange (23) (unknown) Scan parquet spark_catalog.default.store_sales @@ -132,7 +134,7 @@ Join condition: ((((ca_state#14 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND Output [2]: [ss_quantity#4, ss_sold_date_sk#7] Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#13, ca_state#14] -(23) ReusedExchange [Reuses operator id: 33] +(23) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#16] (24) BroadcastHashJoin [codegen id : 5] @@ -152,46 +154,50 @@ Functions [1]: [partial_sum(ss_quantity#4)] Aggregate Attributes [1]: [sum#17] Results [1]: [sum#18] -(27) Exchange +(27) RowToColumnar Input [1]: [sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(28) HashAggregate [codegen id : 6] +(28) CometColumnarExchange +Input [1]: [sum#18] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(29) CometHashAggregate Input [1]: [sum#18] Keys: [] Functions [1]: [sum(ss_quantity#4)] -Aggregate Attributes [1]: [sum(ss_quantity#4)#19] -Results [1]: [sum(ss_quantity#4)#19 AS sum(ss_quantity)#20] + +(30) ColumnarToRow [codegen id : 6] +Input [1]: [sum(ss_quantity)#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_year#21] +Output [2]: [d_date_sk#16, d_year#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(30) CometFilter -Input [2]: [d_date_sk#16, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#16)) +(32) CometFilter +Input [2]: [d_date_sk#16, d_year#20] +Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#16)) -(31) CometProject -Input [2]: [d_date_sk#16, d_year#21] +(33) CometProject +Input [2]: [d_date_sk#16, d_year#20] Arguments: [d_date_sk#16], [d_date_sk#16] -(32) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] -(33) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt index 4022da74f1..fee69a7cc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt @@ -1,50 +1,52 @@ WholeStageCodegen (6) - HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (5) - HashAggregate [ss_quantity] [sum,sum] - Project [ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] - CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometHashAggregate [sum] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [ss_quantity] [sum,sum] + Project [ss_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sold_date_sk] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Project [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 3e14838b85..1327e89162 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -1,78 +1,86 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- Union (70) - :- * Project (23) - : +- * Filter (22) - : +- Window (21) - : +- * Sort (20) - : +- Window (19) - : +- * Sort (18) - : +- Exchange (17) - : +- * HashAggregate (16) - : +- Exchange (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.web_returns (5) - : +- ReusedExchange (11) - :- * Project (46) - : +- * Filter (45) - : +- Window (44) - : +- * Sort (43) - : +- Window (42) - : +- * Sort (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometScan parquet spark_catalog.default.catalog_returns (28) - : +- ReusedExchange (34) - +- * Project (69) - +- * Filter (68) - +- Window (67) - +- * Sort (66) - +- Window (65) - +- * Sort (64) - +- Exchange (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * ColumnarToRow (56) - : +- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometBroadcastExchange (50) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan parquet spark_catalog.default.store_sales (47) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.store_returns (51) - +- ReusedExchange (57) +* ColumnarToRow (82) ++- CometTakeOrderedAndProject (81) + +- CometHashAggregate (80) + +- CometColumnarExchange (79) + +- RowToColumnar (78) + +- * HashAggregate (77) + +- Union (76) + :- * Project (25) + : +- * Filter (24) + : +- Window (23) + : +- * Sort (22) + : +- Window (21) + : +- * ColumnarToRow (20) + : +- CometSort (19) + : +- CometColumnarExchange (18) + : +- CometHashAggregate (17) + : +- CometColumnarExchange (16) + : +- RowToColumnar (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- ReusedExchange (11) + :- * Project (50) + : +- * Filter (49) + : +- Window (48) + : +- * Sort (47) + : +- Window (46) + : +- * ColumnarToRow (45) + : +- CometSort (44) + : +- CometColumnarExchange (43) + : +- CometHashAggregate (42) + : +- CometColumnarExchange (41) + : +- RowToColumnar (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * ColumnarToRow (35) + : : +- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometBroadcastExchange (29) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (26) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (36) + +- * Project (75) + +- * Filter (74) + +- Window (73) + +- * Sort (72) + +- Window (71) + +- * ColumnarToRow (70) + +- CometSort (69) + +- CometColumnarExchange (68) + +- CometHashAggregate (67) + +- CometColumnarExchange (66) + +- RowToColumnar (65) + +- * HashAggregate (64) + +- * Project (63) + +- * BroadcastHashJoin Inner BuildRight (62) + :- * ColumnarToRow (60) + : +- CometProject (59) + : +- CometBroadcastHashJoin (58) + : :- CometBroadcastExchange (54) + : : +- CometProject (53) + : : +- CometFilter (52) + : : +- CometScan parquet spark_catalog.default.store_sales (51) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.store_returns (55) + +- ReusedExchange (61) (unknown) Scan parquet spark_catalog.default.web_sales @@ -122,7 +130,7 @@ Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_re (10) ColumnarToRow [codegen id : 2] Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(11) ReusedExchange [Reuses operator id: 79] +(11) ReusedExchange [Reuses operator id: 87] Output [1]: [d_date_sk#13] (12) BroadcastHashJoin [codegen id : 2] @@ -142,315 +150,331 @@ Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coa Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(15) Exchange +(15) RowToColumnar Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(16) HashAggregate [codegen id : 3] +(16) CometColumnarExchange +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(17) CometHashAggregate Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] -(17) Exchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +(18) CometColumnarExchange +Input [3]: [item#26, return_ratio#27, currency_ratio#28] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(18) Sort [codegen id : 4] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 +(19) CometSort +Input [3]: [item#26, return_ratio#27, currency_ratio#28] +Arguments: [item#26, return_ratio#27, currency_ratio#28], [return_ratio#27 ASC NULLS FIRST] -(19) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] - -(20) Sort [codegen id : 5] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 +(20) ColumnarToRow [codegen id : 3] +Input [3]: [item#26, return_ratio#27, currency_ratio#28] (21) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] +Input [3]: [item#26, return_ratio#27, currency_ratio#28] +Arguments: [rank(return_ratio#27) windowspecdefinition(return_ratio#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#29], [return_ratio#27 ASC NULLS FIRST] + +(22) Sort [codegen id : 4] +Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] +Arguments: [currency_ratio#28 ASC NULLS FIRST], false, 0 -(22) Filter [codegen id : 6] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) +(23) Window +Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] +Arguments: [rank(currency_ratio#28) windowspecdefinition(currency_ratio#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#30], [currency_ratio#28 ASC NULLS FIRST] -(23) Project [codegen id : 6] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +(24) Filter [codegen id : 5] +Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] +Condition : ((return_rank#29 <= 10) OR (currency_rank#30 <= 10)) + +(25) Project [codegen id : 5] +Output [5]: [web AS channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Output [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(25) CometFilter -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) +(27) CometFilter +Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] +Condition : (((((((isnotnull(cs_net_profit#36) AND isnotnull(cs_net_paid#35)) AND isnotnull(cs_quantity#34)) AND (cs_net_profit#36 > 1.00)) AND (cs_net_paid#35 > 0.00)) AND (cs_quantity#34 > 0)) AND isnotnull(cs_order_number#33)) AND isnotnull(cs_item_sk#32)) -(26) CometProject -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(28) CometProject +Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] +Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37], [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] -(27) CometBroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(29) CometBroadcastExchange +Input [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Output [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(29) CometFilter -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) +(31) CometFilter +Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] +Condition : (((isnotnull(cr_return_amount#42) AND (cr_return_amount#42 > 10000.00)) AND isnotnull(cr_order_number#40)) AND isnotnull(cr_item_sk#39)) -(30) CometProject -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +(32) CometProject +Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] +Arguments: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42], [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] -(31) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner +(33) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +Right output [4]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] +Arguments: [cs_order_number#33, cs_item_sk#32], [cr_order_number#40, cr_item_sk#39], Inner -(32) CometProject -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] +(34) CometProject +Input [9]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] +Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] -(33) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] +(35) ColumnarToRow [codegen id : 7] +Input [6]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] -(34) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#48] +(36) ReusedExchange [Reuses operator id: 87] +Output [1]: [d_date_sk#44] -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#48] +(37) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_sold_date_sk#37] +Right keys [1]: [d_date_sk#44] Join type: Inner Join condition: None -(36) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] +(38) Project [codegen id : 7] +Output [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] +Input [7]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42, d_date_sk#44] + +(39) HashAggregate [codegen id : 7] +Input [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] +Keys [1]: [cs_item_sk#32] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#41, 0)), partial_sum(coalesce(cs_quantity#34, 0)), partial_sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] +Results [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] -(37) HashAggregate [codegen id : 8] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] -Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +(40) RowToColumnar +Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] -(38) Exchange -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(41) CometColumnarExchange +Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] +Arguments: hashpartitioning(cs_item_sk#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(39) HashAggregate [codegen id : 9] -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] -Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] +(42) CometHashAggregate +Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] +Keys [1]: [cs_item_sk#32] +Functions [4]: [sum(coalesce(cr_return_quantity#41, 0)), sum(coalesce(cs_quantity#34, 0)), sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] -(40) Exchange -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(43) CometColumnarExchange +Input [3]: [item#57, return_ratio#58, currency_ratio#59] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) Sort [codegen id : 10] -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 +(44) CometSort +Input [3]: [item#57, return_ratio#58, currency_ratio#59] +Arguments: [item#57, return_ratio#58, currency_ratio#59], [return_ratio#58 ASC NULLS FIRST] -(42) Window -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] +(45) ColumnarToRow [codegen id : 8] +Input [3]: [item#57, return_ratio#58, currency_ratio#59] -(43) Sort [codegen id : 11] -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 +(46) Window +Input [3]: [item#57, return_ratio#58, currency_ratio#59] +Arguments: [rank(return_ratio#58) windowspecdefinition(return_ratio#58 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#60], [return_ratio#58 ASC NULLS FIRST] -(44) Window -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] +(47) Sort [codegen id : 9] +Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] +Arguments: [currency_ratio#59 ASC NULLS FIRST], false, 0 -(45) Filter [codegen id : 12] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] -Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) +(48) Window +Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] +Arguments: [rank(currency_ratio#59) windowspecdefinition(currency_ratio#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#61], [currency_ratio#59 ASC NULLS FIRST] -(46) Project [codegen id : 12] -Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +(49) Filter [codegen id : 10] +Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] +Condition : ((return_rank#60 <= 10) OR (currency_rank#61 <= 10)) + +(50) Project [codegen id : 10] +Output [5]: [catalog AS channel#62, item#57, return_ratio#58, return_rank#60, currency_rank#61] +Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] (unknown) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Output [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#68), dynamicpruningexpression(ss_sold_date_sk#68 IN dynamicpruning#69)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(48) CometFilter -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) +(52) CometFilter +Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] +Condition : (((((((isnotnull(ss_net_profit#67) AND isnotnull(ss_net_paid#66)) AND isnotnull(ss_quantity#65)) AND (ss_net_profit#67 > 1.00)) AND (ss_net_paid#66 > 0.00)) AND (ss_quantity#65 > 0)) AND isnotnull(ss_ticket_number#64)) AND isnotnull(ss_item_sk#63)) -(49) CometProject -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(53) CometProject +Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] +Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68], [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] -(50) CometBroadcastExchange -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(54) CometBroadcastExchange +Input [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] (unknown) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Output [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(52) CometFilter -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) +(56) CometFilter +Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] +Condition : (((isnotnull(sr_return_amt#73) AND (sr_return_amt#73 > 10000.00)) AND isnotnull(sr_ticket_number#71)) AND isnotnull(sr_item_sk#70)) -(53) CometProject -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +(57) CometProject +Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] +Arguments: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73], [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] -(54) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner +(58) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +Right output [4]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] +Arguments: [ss_ticket_number#64, ss_item_sk#63], [sr_ticket_number#71, sr_item_sk#70], Inner -(55) CometProject -Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] +(59) CometProject +Input [9]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] +Arguments: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73], [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] -(56) ColumnarToRow [codegen id : 14] -Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] +(60) ColumnarToRow [codegen id : 12] +Input [6]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] -(57) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#83] +(61) ReusedExchange [Reuses operator id: 87] +Output [1]: [d_date_sk#75] -(58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#76] -Right keys [1]: [d_date_sk#83] +(62) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [ss_sold_date_sk#68] +Right keys [1]: [d_date_sk#75] Join type: Inner Join condition: None -(59) Project [codegen id : 14] -Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] - -(60) HashAggregate [codegen id : 14] -Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Keys [1]: [ss_item_sk#71] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] -Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(61) Exchange -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(62) HashAggregate [codegen id : 15] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Keys [1]: [ss_item_sk#71] -Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] -Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] - -(63) Exchange -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] - -(64) Sort [codegen id : 16] -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 - -(65) Window -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] - -(66) Sort [codegen id : 17] -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 - -(67) Window -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] - -(68) Filter [codegen id : 18] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) - -(69) Project [codegen id : 18] -Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] - -(70) Union - -(71) HashAggregate [codegen id : 19] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(63) Project [codegen id : 12] +Output [5]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] +Input [7]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73, d_date_sk#75] + +(64) HashAggregate [codegen id : 12] +Input [5]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] +Keys [1]: [ss_item_sk#63] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#72, 0)), partial_sum(coalesce(ss_quantity#65, 0)), partial_sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#76, sum#77, sum#78, isEmpty#79, sum#80, isEmpty#81] +Results [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(65) RowToColumnar +Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(66) CometColumnarExchange +Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Arguments: hashpartitioning(ss_item_sk#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(67) CometHashAggregate +Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [1]: [ss_item_sk#63] +Functions [4]: [sum(coalesce(sr_return_quantity#72, 0)), sum(coalesce(ss_quantity#65, 0)), sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] + +(68) CometColumnarExchange +Input [3]: [item#88, return_ratio#89, currency_ratio#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(69) CometSort +Input [3]: [item#88, return_ratio#89, currency_ratio#90] +Arguments: [item#88, return_ratio#89, currency_ratio#90], [return_ratio#89 ASC NULLS FIRST] + +(70) ColumnarToRow [codegen id : 13] +Input [3]: [item#88, return_ratio#89, currency_ratio#90] + +(71) Window +Input [3]: [item#88, return_ratio#89, currency_ratio#90] +Arguments: [rank(return_ratio#89) windowspecdefinition(return_ratio#89 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#91], [return_ratio#89 ASC NULLS FIRST] + +(72) Sort [codegen id : 14] +Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] +Arguments: [currency_ratio#90 ASC NULLS FIRST], false, 0 + +(73) Window +Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] +Arguments: [rank(currency_ratio#90) windowspecdefinition(currency_ratio#90 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#92], [currency_ratio#90 ASC NULLS FIRST] + +(74) Filter [codegen id : 15] +Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] +Condition : ((return_rank#91 <= 10) OR (currency_rank#92 <= 10)) + +(75) Project [codegen id : 15] +Output [5]: [store AS channel#93, item#88, return_ratio#89, return_rank#91, currency_rank#92] +Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] + +(76) Union + +(77) HashAggregate [codegen id : 16] +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(72) Exchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(78) RowToColumnar +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(73) HashAggregate [codegen id : 20] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(79) CometColumnarExchange +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Arguments: hashpartitioning(channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(80) CometHashAggregate +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(74) TakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(81) CometTakeOrderedAndProject +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,return_rank#29 ASC NULLS FIRST,currency_rank#30 ASC NULLS FIRST], output=[channel#31,item#26,return_ratio#27,return_rank#29,currency_rank#30]), 100, [channel#31 ASC NULLS FIRST, return_rank#29 ASC NULLS FIRST, currency_rank#30 ASC NULLS FIRST], [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] + +(82) ColumnarToRow [codegen id : 17] +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan parquet spark_catalog.default.date_dim (75) +BroadcastExchange (87) ++- * ColumnarToRow (86) + +- CometProject (85) + +- CometFilter (84) + +- CometScan parquet spark_catalog.default.date_dim (83) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#106, d_moy#107] +Output [3]: [d_date_sk#13, d_year#94, d_moy#95] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] -Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) +(84) CometFilter +Input [3]: [d_date_sk#13, d_year#94, d_moy#95] +Condition : ((((isnotnull(d_year#94) AND isnotnull(d_moy#95)) AND (d_year#94 = 2001)) AND (d_moy#95 = 12)) AND isnotnull(d_date_sk#13)) -(77) CometProject -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +(85) CometProject +Input [3]: [d_date_sk#13, d_year#94, d_moy#95] Arguments: [d_date_sk#13], [d_date_sk#13] -(78) ColumnarToRow [codegen id : 1] +(86) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(79) BroadcastExchange +(87) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#68 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index 43ebf34cc0..6ea9407566 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -1,121 +1,123 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (20) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (19) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - Sort [return_ratio] - InputAdapter - Exchange #2 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (12) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (11) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (10) - Sort [return_ratio] - InputAdapter - Exchange #6 - WholeStageCodegen (9) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #7 - WholeStageCodegen (8) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #8 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (18) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (17) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (16) - Sort [return_ratio] - InputAdapter - Exchange #9 - WholeStageCodegen (15) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #11 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 +WholeStageCodegen (17) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (5) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (4) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometSort [return_ratio] + CometColumnarExchange #2 + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [ws_item_sk] #3 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (10) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometSort [return_ratio] + CometColumnarExchange #6 + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [cs_item_sk] #7 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #8 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (15) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (14) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (13) + ColumnarToRow + InputAdapter + CometSort [return_ratio] + CometColumnarExchange #9 + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [ss_item_sk] #10 + RowToColumnar + WholeStageCodegen (12) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #11 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 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 10ba5439e7..3f204c402c 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,74 +1,82 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Expand (66) - +- Union (65) - :- * HashAggregate (20) - : +- Exchange (19) - : +- * HashAggregate (18) - : +- * Project (17) - : +- * BroadcastHashJoin Inner BuildRight (16) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (8) - : : : +- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : +- ReusedExchange (9) - : +- BroadcastExchange (15) - : +- * ColumnarToRow (14) - : +- CometFilter (13) - : +- CometScan parquet spark_catalog.default.store (12) - :- * HashAggregate (40) - : +- Exchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- CometUnion (27) - : : : :- CometProject (23) - : : : : +- CometFilter (22) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (24) - : : +- ReusedExchange (29) - : +- BroadcastExchange (35) - : +- * ColumnarToRow (34) - : +- CometFilter (33) - : +- CometScan parquet spark_catalog.default.catalog_page (32) - +- * HashAggregate (64) - +- Exchange (63) - +- * HashAggregate (62) - +- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * Project (55) - : +- * BroadcastHashJoin Inner BuildRight (54) - : :- * ColumnarToRow (52) - : : +- CometUnion (51) - : : :- CometProject (43) - : : : +- CometFilter (42) - : : : +- CometScan parquet spark_catalog.default.web_sales (41) - : : +- CometProject (50) - : : +- CometBroadcastHashJoin (49) - : : :- CometBroadcastExchange (45) - : : : +- CometScan parquet spark_catalog.default.web_returns (44) - : : +- CometProject (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.web_sales (46) - : +- ReusedExchange (53) - +- BroadcastExchange (59) - +- * ColumnarToRow (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.web_site (56) +* ColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- RowToColumnar (74) + +- * HashAggregate (73) + +- * Expand (72) + +- Union (71) + :- * HashAggregate (22) + : +- * ColumnarToRow (21) + : +- CometColumnarExchange (20) + : +- RowToColumnar (19) + : +- * HashAggregate (18) + : +- * Project (17) + : +- * BroadcastHashJoin Inner BuildRight (16) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (8) + : : : +- CometUnion (7) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : +- ReusedExchange (9) + : +- BroadcastExchange (15) + : +- * ColumnarToRow (14) + : +- CometFilter (13) + : +- CometScan parquet spark_catalog.default.store (12) + :- * HashAggregate (44) + : +- * ColumnarToRow (43) + : +- CometColumnarExchange (42) + : +- RowToColumnar (41) + : +- * HashAggregate (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (33) + : : +- * BroadcastHashJoin Inner BuildRight (32) + : : :- * ColumnarToRow (30) + : : : +- CometUnion (29) + : : : :- CometProject (25) + : : : : +- CometFilter (24) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (23) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) + : : +- ReusedExchange (31) + : +- BroadcastExchange (37) + : +- * ColumnarToRow (36) + : +- CometFilter (35) + : +- CometScan parquet spark_catalog.default.catalog_page (34) + +- * HashAggregate (70) + +- * ColumnarToRow (69) + +- CometColumnarExchange (68) + +- RowToColumnar (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (59) + : +- * BroadcastHashJoin Inner BuildRight (58) + : :- * ColumnarToRow (56) + : : +- CometUnion (55) + : : :- CometProject (47) + : : : +- CometFilter (46) + : : : +- CometScan parquet spark_catalog.default.web_sales (45) + : : +- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastExchange (49) + : : : +- CometScan parquet spark_catalog.default.web_returns (48) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan parquet spark_catalog.default.web_sales (50) + : +- ReusedExchange (57) + +- BroadcastExchange (63) + +- * ColumnarToRow (62) + +- CometFilter (61) + +- CometScan parquet spark_catalog.default.web_site (60) (unknown) Scan parquet spark_catalog.default.store_sales @@ -110,7 +118,7 @@ Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_a (8) ColumnarToRow [codegen id : 3] Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -(9) ReusedExchange [Reuses operator id: 75] +(9) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#22] (10) BroadcastHashJoin [codegen id : 3] @@ -158,11 +166,17 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -(19) Exchange +(19) RowToColumnar Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(20) HashAggregate [codegen id : 4] +(20) CometColumnarExchange +Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(21) ColumnarToRow [codegen id : 4] +Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] + +(22) HashAggregate [codegen id : 4] Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] Keys [1]: [s_store_id#24] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] @@ -177,11 +191,11 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(22) CometFilter +(24) CometFilter Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Condition : isnotnull(cs_catalog_page_sk#42) -(23) CometProject +(25) CometProject Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Arguments: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52], [cs_catalog_page_sk#42 AS page_sk#47, cs_sold_date_sk#45 AS date_sk#48, cs_ext_sales_price#43 AS sales_price#49, cs_net_profit#44 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] @@ -193,31 +207,31 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(25) CometFilter +(27) CometFilter Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Condition : isnotnull(cr_catalog_page_sk#53) -(26) CometProject +(28) CometProject Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Arguments: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62], [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] -(27) CometUnion +(29) CometUnion Child 0 Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62] -(28) ColumnarToRow [codegen id : 7] +(30) ColumnarToRow [codegen id : 7] Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] -(29) ReusedExchange [Reuses operator id: 75] +(31) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#63] -(30) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 7] Left keys [1]: [date_sk#48] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(31) Project [codegen id : 7] +(33) Project [codegen id : 7] Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] @@ -228,39 +242,45 @@ Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(33) CometFilter +(35) CometFilter Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Condition : isnotnull(cp_catalog_page_sk#64) -(34) ColumnarToRow [codegen id : 6] +(36) ColumnarToRow [codegen id : 6] Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -(35) BroadcastExchange +(37) BroadcastExchange Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(36) BroadcastHashJoin [codegen id : 7] +(38) BroadcastHashJoin [codegen id : 7] Left keys [1]: [page_sk#47] Right keys [1]: [cp_catalog_page_sk#64] Join type: Inner Join condition: None -(37) Project [codegen id : 7] +(39) Project [codegen id : 7] Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#65] -(38) HashAggregate [codegen id : 7] +(40) HashAggregate [codegen id : 7] Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] Aggregate Attributes [4]: [sum#66, sum#67, sum#68, sum#69] Results [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -(39) Exchange +(41) RowToColumnar +Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] + +(42) CometColumnarExchange +Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(43) ColumnarToRow [codegen id : 8] Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(40) HashAggregate [codegen id : 8] +(44) HashAggregate [codegen id : 8] Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -275,11 +295,11 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(42) CometFilter +(46) CometFilter Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Condition : isnotnull(ws_web_site_sk#83) -(43) CometProject +(47) CometProject Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] @@ -290,7 +310,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] ReadSchema: struct -(45) CometBroadcastExchange +(49) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] @@ -301,40 +321,40 @@ 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 -(47) CometFilter +(51) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(48) CometProject +(52) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(49) CometBroadcastHashJoin +(53) CometBroadcastHashJoin Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(50) CometProject +(54) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -(51) CometUnion +(55) CometUnion Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] -(52) ColumnarToRow [codegen id : 11] +(56) ColumnarToRow [codegen id : 11] Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(53) ReusedExchange [Reuses operator id: 75] +(57) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#109] -(54) BroadcastHashJoin [codegen id : 11] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(55) Project [codegen id : 11] +(59) Project [codegen id : 11] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -345,113 +365,123 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(57) CometFilter +(61) CometFilter Input [2]: [web_site_sk#110, web_site_id#111] Condition : isnotnull(web_site_sk#110) -(58) ColumnarToRow [codegen id : 10] +(62) ColumnarToRow [codegen id : 10] Input [2]: [web_site_sk#110, web_site_id#111] -(59) BroadcastExchange +(63) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#111] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(60) BroadcastHashJoin [codegen id : 11] +(64) BroadcastHashJoin [codegen id : 11] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(61) Project [codegen id : 11] +(65) Project [codegen id : 11] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(62) HashAggregate [codegen id : 11] +(66) HashAggregate [codegen id : 11] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -(63) Exchange +(67) RowToColumnar Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(64) HashAggregate [codegen id : 12] +(68) CometColumnarExchange +Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(69) ColumnarToRow [codegen id : 12] +Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] + +(70) HashAggregate [codegen id : 12] Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#126, web channel AS channel#127, concat(web_site, web_site_id#111) AS id#128] -(65) Union +(71) Union -(66) Expand [codegen id : 13] +(72) Expand [codegen id : 13] Input [5]: [sales#37, returns#38, profit#39, channel#40, id#41] Arguments: [[sales#37, returns#38, profit#39, channel#40, id#41, 0], [sales#37, returns#38, profit#39, channel#40, null, 1], [sales#37, returns#38, profit#39, null, null, 3]], [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] -(67) HashAggregate [codegen id : 13] +(73) HashAggregate [codegen id : 13] Input [6]: [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] Keys [3]: [channel#129, id#130, spark_grouping_id#131] Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] Results [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -(68) Exchange +(74) RowToColumnar Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#129, id#130, spark_grouping_id#131, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(69) HashAggregate [codegen id : 14] +(75) CometColumnarExchange +Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Arguments: hashpartitioning(channel#129, id#130, spark_grouping_id#131, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(76) CometHashAggregate Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Keys [3]: [channel#129, id#130, spark_grouping_id#131] Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#144, sum(returns#38)#145, sum(profit#39)#146] -Results [5]: [channel#129, id#130, sum(sales#37)#144 AS sales#147, sum(returns#38)#145 AS returns#148, sum(profit#39)#146 AS profit#149] -(70) TakeOrderedAndProject -Input [5]: [channel#129, id#130, sales#147, returns#148, profit#149] -Arguments: 100, [channel#129 ASC NULLS FIRST, id#130 ASC NULLS FIRST], [channel#129, id#130, sales#147, returns#148, profit#149] +(77) CometTakeOrderedAndProject +Input [5]: [channel#129, id#130, sales#144, returns#145, profit#146] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#129 ASC NULLS FIRST,id#130 ASC NULLS FIRST], output=[channel#129,id#130,sales#144,returns#145,profit#146]), 100, [channel#129 ASC NULLS FIRST, id#130 ASC NULLS FIRST], [channel#129, id#130, sales#144, returns#145, profit#146] + +(78) ColumnarToRow [codegen id : 14] +Input [5]: [channel#129, id#130, sales#144, returns#145, profit#146] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (83) ++- * ColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan parquet spark_catalog.default.date_dim (79) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#150] +Output [2]: [d_date_sk#22, d_date#147] 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 -(72) CometFilter -Input [2]: [d_date_sk#22, d_date#150] -Condition : (((isnotnull(d_date#150) AND (d_date#150 >= 2000-08-23)) AND (d_date#150 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) +(80) CometFilter +Input [2]: [d_date_sk#22, d_date#147] +Condition : (((isnotnull(d_date#147) AND (d_date#147 >= 2000-08-23)) AND (d_date#147 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(73) CometProject -Input [2]: [d_date_sk#22, d_date#150] +(81) CometProject +Input [2]: [d_date_sk#22, d_date#147] Arguments: [d_date_sk#22], [d_date_sk#22] -(74) ColumnarToRow [codegen id : 1] +(82) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(75) BroadcastExchange +(83) 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 = 21 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 48 Hosting Expression = wr_returned_date_sk#98 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 3d539d5917..daa36d5049 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,111 +1,119 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (14) - 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] - InputAdapter - Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (13) - 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 (4) - 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] - InputAdapter - Exchange [s_store_id] #2 - WholeStageCodegen (3) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan 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] - CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow +WholeStageCodegen (14) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (13) + 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 (4) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan 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] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (8) - 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] - InputAdapter - Exchange [cp_catalog_page_id] #5 - WholeStageCodegen (7) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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] - CometScan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #5 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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] + CometScan 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 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (12) - 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] - InputAdapter - Exchange [web_site_id] #7 - WholeStageCodegen (11) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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,ws_item_sk,ws_order_number] - CometBroadcastExchange #8 - CometScan 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_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (12) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #7 + RowToColumnar + WholeStageCodegen (11) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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,ws_item_sk,ws_order_number] + CometBroadcastExchange #8 + CometScan 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_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + BroadcastExchange #9 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan 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/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt index b178faed1a..c30a1571cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt @@ -1,32 +1,34 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Project (24) - +- * BroadcastHashJoin Inner BuildRight (23) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.date_dim (16) - +- ReusedExchange (22) +* ColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometHashAggregate (28) + +- CometColumnarExchange (27) + +- RowToColumnar (26) + +- * HashAggregate (25) + +- * Project (24) + +- * BroadcastHashJoin Inner BuildRight (23) + :- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store (10) + : +- BroadcastExchange (19) + : +- * ColumnarToRow (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.date_dim (16) + +- ReusedExchange (22) (unknown) Scan parquet spark_catalog.default.store_sales @@ -129,7 +131,7 @@ Join condition: None Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#22] -(22) ReusedExchange [Reuses operator id: 33] +(22) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#23] (23) BroadcastHashJoin [codegen id : 5] @@ -149,50 +151,54 @@ Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk# Aggregate Attributes [5]: [sum#24, sum#25, sum#26, sum#27, sum#28] Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] -(26) Exchange +(26) RowToColumnar Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(27) HashAggregate [codegen id : 6] +(27) CometColumnarExchange +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(28) CometHashAggregate Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34 AS 30 days #39, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35 AS 31 - 60 days #40, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36 AS 61 - 90 days #41, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37 AS 91 - 120 days #42, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38 AS >120 days #43] -(28) TakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] -Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] +(29) CometTakeOrderedAndProject +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #34, 31 - 60 days #35, 61 - 90 days #36, 91 - 120 days #37, >120 days #38] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#12 ASC NULLS FIRST,s_company_id#13 ASC NULLS FIRST,s_street_number#14 ASC NULLS FIRST,s_street_name#15 ASC NULLS FIRST,s_street_type#16 ASC NULLS FIRST,s_suite_number#17 ASC NULLS FIRST,s_city#18 ASC NULLS FIRST,s_county#19 ASC NULLS FIRST,s_state#20 ASC NULLS FIRST,s_zip#21 ASC NULLS FIRST], output=[s_store_name#12,s_company_id#13,s_street_number#14,s_street_name#15,s_street_type#16,s_suite_number#17,s_city#18,s_county#19,s_state#20,s_zip#21,30 days #34,31 - 60 days #35,61 - 90 days #36,91 - 120 days #37,>120 days #38]), 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #34, 31 - 60 days #35, 61 - 90 days #36, 91 - 120 days #37, >120 days #38] + +(30) ColumnarToRow [codegen id : 6] +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #34, 31 - 60 days #35, 61 - 90 days #36, 91 - 120 days #37, >120 days #38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#23, d_year#44, d_moy#45] +Output [3]: [d_date_sk#23, d_year#39, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] ReadSchema: struct -(30) CometFilter -Input [3]: [d_date_sk#23, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 8)) AND isnotnull(d_date_sk#23)) +(32) CometFilter +Input [3]: [d_date_sk#23, d_year#39, d_moy#40] +Condition : ((((isnotnull(d_year#39) AND isnotnull(d_moy#40)) AND (d_year#39 = 2001)) AND (d_moy#40 = 8)) AND isnotnull(d_date_sk#23)) -(31) CometProject -Input [3]: [d_date_sk#23, d_year#44, d_moy#45] +(33) CometProject +Input [3]: [d_date_sk#23, d_year#39, d_moy#40] Arguments: [d_date_sk#23], [d_date_sk#23] -(32) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#23] -(33) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt index dfdcaf4975..07cfd15950 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt @@ -1,50 +1,52 @@ -TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) - HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - InputAdapter - Exchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 - WholeStageCodegen (5) - HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] +WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] + CometColumnarExchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt index c55b122617..68387ead7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt @@ -1,41 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (37) -+- * Filter (36) - +- Window (35) - +- * Sort (34) - +- Exchange (33) - +- * Project (32) - +- * SortMergeJoin FullOuter (31) - :- * Sort (15) - : +- Exchange (14) - : +- * Project (13) - : +- Window (12) - : +- * Sort (11) - : +- Exchange (10) - : +- * HashAggregate (9) - : +- Exchange (8) - : +- * HashAggregate (7) - : +- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- * Sort (30) - +- Exchange (29) - +- * Project (28) - +- Window (27) - +- * Sort (26) - +- Exchange (25) - +- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.store_sales (16) - +- ReusedExchange (19) +TakeOrderedAndProject (44) ++- * Filter (43) + +- Window (42) + +- * ColumnarToRow (41) + +- CometSort (40) + +- CometColumnarExchange (39) + +- CometProject (38) + +- CometSortMergeJoin (37) + :- CometSort (18) + : +- CometColumnarExchange (17) + : +- RowToColumnar (16) + : +- * Project (15) + : +- Window (14) + : +- * ColumnarToRow (13) + : +- CometSort (12) + : +- CometColumnarExchange (11) + : +- CometHashAggregate (10) + : +- CometColumnarExchange (9) + : +- RowToColumnar (8) + : +- * HashAggregate (7) + : +- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- ReusedExchange (4) + +- CometSort (36) + +- CometColumnarExchange (35) + +- RowToColumnar (34) + +- * Project (33) + +- Window (32) + +- * ColumnarToRow (31) + +- CometSort (30) + +- CometColumnarExchange (29) + +- CometHashAggregate (28) + +- CometColumnarExchange (27) + +- RowToColumnar (26) + +- * HashAggregate (25) + +- * Project (24) + +- * BroadcastHashJoin Inner BuildRight (23) + :- * ColumnarToRow (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.store_sales (19) + +- ReusedExchange (22) (unknown) Scan parquet spark_catalog.default.web_sales @@ -53,7 +60,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 2] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 42] +(4) ReusedExchange [Reuses operator id: 49] Output [2]: [d_date_sk#5, d_date#6] (5) BroadcastHashJoin [codegen id : 2] @@ -73,173 +80,189 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] Aggregate Attributes [1]: [sum#7] Results [3]: [ws_item_sk#1, d_date#6, sum#8] -(8) Exchange +(8) RowToColumnar Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) HashAggregate [codegen id : 3] +(9) CometColumnarExchange +Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(10) CometHashAggregate Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS _w0#11, ws_item_sk#1] -(10) Exchange -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(11) CometColumnarExchange +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(12) CometSort +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(11) Sort [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 +(13) ColumnarToRow [codegen id : 3] +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -(12) Window -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(14) Window +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(13) Project [codegen id : 5] -Output [3]: [item_sk#10, d_date#6, cume_sales#12] -Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] +(15) Project [codegen id : 4] +Output [3]: [item_sk#9, d_date#6, cume_sales#11] +Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] -(14) Exchange -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(16) RowToColumnar +Input [3]: [item_sk#9, d_date#6, cume_sales#11] -(15) Sort [codegen id : 6] -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 +(17) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#11] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(18) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#11] +Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(17) CometFilter -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_item_sk#13) +(20) CometFilter +Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] +Condition : isnotnull(ss_item_sk#12) -(18) ColumnarToRow [codegen id : 8] -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +(21) ColumnarToRow [codegen id : 6] +Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -(19) ReusedExchange [Reuses operator id: 42] -Output [2]: [d_date_sk#17, d_date#18] +(22) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#16, d_date#17] -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] +(23) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#14] +Right keys [1]: [d_date_sk#16] Join type: Inner Join condition: None -(21) Project [codegen id : 8] -Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] -Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] - -(22) HashAggregate [codegen id : 8] -Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] -Keys [2]: [ss_item_sk#13, d_date#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [ss_item_sk#13, d_date#18, sum#20] - -(23) Exchange -Input [3]: [ss_item_sk#13, d_date#18, sum#20] -Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(24) HashAggregate [codegen id : 9] -Input [3]: [ss_item_sk#13, d_date#18, sum#20] -Keys [2]: [ss_item_sk#13, d_date#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#21] -Results [4]: [ss_item_sk#13 AS item_sk#22, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#21,17,2) AS _w0#23, ss_item_sk#13] - -(25) Exchange -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(26) Sort [codegen id : 10] -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -Arguments: [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 - -(27) Window -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -Arguments: [sum(_w0#23) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#24], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] - -(28) Project [codegen id : 11] -Output [3]: [item_sk#22, d_date#18, cume_sales#24] -Input [5]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13, cume_sales#24] - -(29) Exchange -Input [3]: [item_sk#22, d_date#18, cume_sales#24] -Arguments: hashpartitioning(item_sk#22, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(30) Sort [codegen id : 12] -Input [3]: [item_sk#22, d_date#18, cume_sales#24] -Arguments: [item_sk#22 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 - -(31) SortMergeJoin [codegen id : 13] -Left keys [2]: [item_sk#10, d_date#6] -Right keys [2]: [item_sk#22, d_date#18] -Join type: FullOuter -Join condition: None +(24) Project [codegen id : 6] +Output [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] +Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] + +(25) HashAggregate [codegen id : 6] +Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] +Keys [2]: [ss_item_sk#12, d_date#17] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] +Aggregate Attributes [1]: [sum#18] +Results [3]: [ss_item_sk#12, d_date#17, sum#19] + +(26) RowToColumnar +Input [3]: [ss_item_sk#12, d_date#17, sum#19] + +(27) CometColumnarExchange +Input [3]: [ss_item_sk#12, d_date#17, sum#19] +Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(28) CometHashAggregate +Input [3]: [ss_item_sk#12, d_date#17, sum#19] +Keys [2]: [ss_item_sk#12, d_date#17] +Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] + +(29) CometColumnarExchange +Input [4]: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12] +Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(30) CometSort +Input [4]: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12] +Arguments: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] + +(31) ColumnarToRow [codegen id : 7] +Input [4]: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12] + +(32) Window +Input [4]: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12] +Arguments: [sum(_w0#21) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#22], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] + +(33) Project [codegen id : 8] +Output [3]: [item_sk#20, d_date#17, cume_sales#22] +Input [5]: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12, cume_sales#22] + +(34) RowToColumnar +Input [3]: [item_sk#20, d_date#17, cume_sales#22] + +(35) CometColumnarExchange +Input [3]: [item_sk#20, d_date#17, cume_sales#22] +Arguments: hashpartitioning(item_sk#20, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(36) CometSort +Input [3]: [item_sk#20, d_date#17, cume_sales#22] +Arguments: [item_sk#20, d_date#17, cume_sales#22], [item_sk#20 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] + +(37) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#11] +Right output [3]: [item_sk#20, d_date#17, cume_sales#22] +Arguments: [item_sk#9, d_date#6], [item_sk#20, d_date#17], FullOuter + +(38) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#20, d_date#17, cume_sales#22] +Arguments: [item_sk#23, d_date#24, web_sales#25, store_sales#26], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#20 END AS item_sk#23, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#24, cume_sales#11 AS web_sales#25, cume_sales#22 AS store_sales#26] -(32) Project [codegen id : 13] -Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#22 END AS item_sk#25, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#26, cume_sales#12 AS web_sales#27, cume_sales#24 AS store_sales#28] -Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#22, d_date#18, cume_sales#24] +(39) CometColumnarExchange +Input [4]: [item_sk#23, d_date#24, web_sales#25, store_sales#26] +Arguments: hashpartitioning(item_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(33) Exchange -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: hashpartitioning(item_sk#25, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(40) CometSort +Input [4]: [item_sk#23, d_date#24, web_sales#25, store_sales#26] +Arguments: [item_sk#23, d_date#24, web_sales#25, store_sales#26], [item_sk#23 ASC NULLS FIRST, d_date#24 ASC NULLS FIRST] -(34) Sort [codegen id : 14] -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], false, 0 +(41) ColumnarToRow [codegen id : 9] +Input [4]: [item_sk#23, d_date#24, web_sales#25, store_sales#26] -(35) Window -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: [max(web_sales#27) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#29, max(store_sales#28) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#30], [item_sk#25], [d_date#26 ASC NULLS FIRST] +(42) Window +Input [4]: [item_sk#23, d_date#24, web_sales#25, store_sales#26] +Arguments: [max(web_sales#25) windowspecdefinition(item_sk#23, d_date#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#27, max(store_sales#26) windowspecdefinition(item_sk#23, d_date#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#28], [item_sk#23], [d_date#24 ASC NULLS FIRST] -(36) Filter [codegen id : 15] -Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] -Condition : ((isnotnull(web_cumulative#29) AND isnotnull(store_cumulative#30)) AND (web_cumulative#29 > store_cumulative#30)) +(43) Filter [codegen id : 10] +Input [6]: [item_sk#23, d_date#24, web_sales#25, store_sales#26, web_cumulative#27, store_cumulative#28] +Condition : ((isnotnull(web_cumulative#27) AND isnotnull(store_cumulative#28)) AND (web_cumulative#27 > store_cumulative#28)) -(37) TakeOrderedAndProject -Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] -Arguments: 100, [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] +(44) TakeOrderedAndProject +Input [6]: [item_sk#23, d_date#24, web_sales#25, store_sales#26, web_cumulative#27, store_cumulative#28] +Arguments: 100, [item_sk#23 ASC NULLS FIRST, d_date#24 ASC NULLS FIRST], [item_sk#23, d_date#24, web_sales#25, store_sales#26, web_cumulative#27, store_cumulative#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (42) -+- * ColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan parquet spark_catalog.default.date_dim (38) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#31] +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(39) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] -Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1200)) AND (d_month_seq#31 <= 1211)) AND isnotnull(d_date_sk#5)) +(46) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#29] +Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#5)) -(40) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] +(47) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#29] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(41) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(42) BroadcastExchange +(49) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index 181cd1b98c..d46e28b44c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -1,75 +1,71 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (15) + WholeStageCodegen (10) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (14) - Sort [item_sk,d_date] + WholeStageCodegen (9) + ColumnarToRow InputAdapter - Exchange [item_sk] #1 - WholeStageCodegen (13) - Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] - SortMergeJoin [item_sk,d_date,item_sk,d_date] - InputAdapter - WholeStageCodegen (6) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (5) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (4) - Sort [ws_item_sk,d_date] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - InputAdapter - Exchange [ws_item_sk,d_date] #4 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (12) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #6 - WholeStageCodegen (11) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (10) - Sort [ss_item_sk,d_date] - InputAdapter - Exchange [ss_item_sk] #7 - WholeStageCodegen (9) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - InputAdapter - Exchange [ss_item_sk,d_date] #8 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + CometSort [item_sk,d_date] + CometColumnarExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,item_sk,d_date] + CometSort [item_sk,d_date] + CometColumnarExchange [item_sk,d_date] #2 + RowToColumnar + WholeStageCodegen (4) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometSort [ws_item_sk,d_date] + CometColumnarExchange [ws_item_sk] #3 + CometHashAggregate [ws_item_sk,d_date,sum] + CometColumnarExchange [ws_item_sk,d_date] #4 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + CometSort [item_sk,d_date] + CometColumnarExchange [item_sk,d_date] #6 + RowToColumnar + WholeStageCodegen (8) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometSort [ss_item_sk,d_date] + CometColumnarExchange [ss_item_sk] #7 + CometHashAggregate [ss_item_sk,d_date,sum] + CometColumnarExchange [ss_item_sk,d_date] #8 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt index 8df157e51b..d6e0e7ab24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt @@ -1,25 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +* ColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometHashAggregate (21) + +- CometColumnarExchange (20) + +- RowToColumnar (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) (unknown) Scan parquet spark_catalog.default.date_dim @@ -108,18 +110,22 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -(19) Exchange +(19) RowToColumnar Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) HashAggregate [codegen id : 4] +(20) CometColumnarExchange +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(21) CometHashAggregate Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(21) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#14, brand#15, ext_price#16] -Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, ext_price#16] +(22) CometTakeOrderedAndProject +Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), 100, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] + +(23) ColumnarToRow [codegen id : 4] +Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt index 91fdc2f17d..7607d1ccdd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt @@ -1,31 +1,33 @@ -TakeOrderedAndProject [d_year,ext_price,brand_id,brand] - WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - InputAdapter - Exchange [d_year,i_brand,i_brand_id] #1 - WholeStageCodegen (3) - HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,ext_price,brand_id,brand] + CometHashAggregate [d_year,i_brand,i_brand_id,sum] + CometColumnarExchange [d_year,i_brand,i_brand_id] #1 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 5142c9e4e4..5cb48513fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -1,32 +1,34 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * Project (27) - +- * Filter (26) - +- Window (25) - +- * Sort (24) - +- Exchange (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.store (14) +TakeOrderedAndProject (30) ++- * Project (29) + +- * Filter (28) + +- Window (27) + +- * ColumnarToRow (26) + +- CometSort (25) + +- CometColumnarExchange (24) + +- CometHashAggregate (23) + +- CometColumnarExchange (22) + +- RowToColumnar (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.store_sales (5) + : +- ReusedExchange (11) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) (unknown) Scan parquet spark_catalog.default.item @@ -76,7 +78,7 @@ Join condition: None Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -(11) ReusedExchange [Reuses operator id: 33] +(11) ReusedExchange [Reuses operator id: 35] Output [2]: [d_date_sk#15, d_qoy#16] (12) BroadcastHashJoin [codegen id : 4] @@ -124,70 +126,74 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] Aggregate Attributes [1]: [sum#18] Results [3]: [i_manufact_id#5, d_qoy#16, sum#19] -(21) Exchange +(21) RowToColumnar Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) HashAggregate [codegen id : 5] +(22) CometColumnarExchange +Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] +Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(23) CometHashAggregate Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] Keys [2]: [i_manufact_id#5, d_qoy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] -(23) Exchange -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(24) CometColumnarExchange +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(25) CometSort +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] -(24) Sort [codegen id : 6] -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [i_manufact_id#5 ASC NULLS FIRST], false, 0 +(26) ColumnarToRow [codegen id : 5] +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -(25) Window -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] +(27) Window +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] -(26) Filter [codegen id : 7] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END +(28) Filter [codegen id : 6] +Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] +Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END -(27) Project [codegen id : 7] -Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] +(29) Project [codegen id : 6] +Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -(28) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +(30) TakeOrderedAndProject +Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] +Output [3]: [d_date_sk#15, d_month_seq#23, d_qoy#16] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(30) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) +(32) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#23, d_qoy#16] +Condition : (d_month_seq#23 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(31) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] +(33) CometProject +Input [3]: [d_date_sk#15, d_month_seq#23, d_qoy#16] Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] -(32) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#16] -(33) BroadcastExchange +(35) BroadcastExchange Input [2]: [d_date_sk#15, d_qoy#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index adda5c34f7..af0269ae39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -1,17 +1,17 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (7) + WholeStageCodegen (6) Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] - WholeStageCodegen (6) - Sort [i_manufact_id] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [i_manufact_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_manufact_id,d_qoy] #2 + CometSort [i_manufact_id] + CometColumnarExchange [i_manufact_id] #1 + CometHashAggregate [i_manufact_id,d_qoy,sum] + CometColumnarExchange [i_manufact_id,d_qoy] #2 + RowToColumnar WholeStageCodegen (4) HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] Project [i_manufact_id,ss_sales_price,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 00c8f9c9b0..5baa553cf1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -1,59 +1,63 @@ == Physical Plan == -TakeOrderedAndProject (55) -+- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (33) - : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : :- * HashAggregate (27) - : : : : +- Exchange (26) - : : : : +- * HashAggregate (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * ColumnarToRow (8) - : : : : : : : +- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (4) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- CometProject (11) - : : : : : : +- CometFilter (10) - : : : : : : +- CometScan parquet spark_catalog.default.item (9) - : : : : : +- ReusedExchange (16) - : : : : +- BroadcastExchange (22) - : : : : +- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (31) - : : : +- * ColumnarToRow (30) - : : : +- CometFilter (29) - : : : +- CometScan parquet spark_catalog.default.store_sales (28) - : : +- BroadcastExchange (37) - : : +- * ColumnarToRow (36) - : : +- CometFilter (35) - : : +- CometScan parquet spark_catalog.default.customer_address (34) - : +- BroadcastExchange (43) - : +- * ColumnarToRow (42) - : +- CometFilter (41) - : +- CometScan parquet spark_catalog.default.store (40) - +- ReusedExchange (46) +* ColumnarToRow (59) ++- CometTakeOrderedAndProject (58) + +- CometHashAggregate (57) + +- CometColumnarExchange (56) + +- CometHashAggregate (55) + +- CometHashAggregate (54) + +- CometColumnarExchange (53) + +- RowToColumnar (52) + +- * HashAggregate (51) + +- * Project (50) + +- * BroadcastHashJoin Inner BuildRight (49) + :- * Project (47) + : +- * BroadcastHashJoin Inner BuildRight (46) + : :- * Project (41) + : : +- * BroadcastHashJoin Inner BuildRight (40) + : : :- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (29) + : : : : +- CometHashAggregate (28) + : : : : +- CometColumnarExchange (27) + : : : : +- RowToColumnar (26) + : : : : +- * HashAggregate (25) + : : : : +- * Project (24) + : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * ColumnarToRow (8) + : : : : : : : +- CometUnion (7) + : : : : : : : :- CometProject (3) + : : : : : : : : +- CometFilter (2) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : +- CometProject (6) + : : : : : : : +- CometFilter (5) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (4) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * ColumnarToRow (12) + : : : : : : +- CometProject (11) + : : : : : : +- CometFilter (10) + : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : +- ReusedExchange (16) + : : : : +- BroadcastExchange (22) + : : : : +- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (33) + : : : +- * ColumnarToRow (32) + : : : +- CometFilter (31) + : : : +- CometScan parquet spark_catalog.default.store_sales (30) + : : +- BroadcastExchange (39) + : : +- * ColumnarToRow (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.customer_address (36) + : +- BroadcastExchange (45) + : +- * ColumnarToRow (44) + : +- CometFilter (43) + : +- CometScan parquet spark_catalog.default.store (42) + +- ReusedExchange (48) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -127,7 +131,7 @@ Join condition: None Output [2]: [sold_date_sk#5, customer_sk#6] Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] -(16) ReusedExchange [Reuses operator id: 60] +(16) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#17] (17) BroadcastHashJoin [codegen id : 4] @@ -175,16 +179,20 @@ Functions: [] Aggregate Attributes: [] Results [2]: [c_customer_sk#18, c_current_addr_sk#19] -(26) Exchange +(26) RowToColumnar Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: hashpartitioning(c_customer_sk#18, c_current_addr_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(27) HashAggregate [codegen id : 9] +(27) CometColumnarExchange +Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +Arguments: hashpartitioning(c_customer_sk#18, c_current_addr_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(28) CometHashAggregate Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] Functions: [] -Aggregate Attributes: [] -Results [2]: [c_customer_sk#18, c_current_addr_sk#19] + +(29) ColumnarToRow [codegen id : 9] +Input [2]: [c_customer_sk#18, c_current_addr_sk#19] (unknown) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] @@ -194,24 +202,24 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(29) CometFilter +(31) CometFilter Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) -(30) ColumnarToRow [codegen id : 5] +(32) ColumnarToRow [codegen id : 5] Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -(31) BroadcastExchange +(33) BroadcastExchange Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#18] Right keys [1]: [ss_customer_sk#20] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(35) Project [codegen id : 9] Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] @@ -222,24 +230,24 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county), IsNotNull(ca_state)] ReadSchema: struct -(35) CometFilter +(37) CometFilter Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnotnull(ca_state#26)) -(36) ColumnarToRow [codegen id : 6] +(38) ColumnarToRow [codegen id : 6] Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -(37) BroadcastExchange +(39) BroadcastExchange Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#19] Right keys [1]: [ca_address_sk#24] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(41) Project [codegen id : 9] Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26] Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#26] @@ -250,226 +258,226 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] ReadSchema: struct -(41) CometFilter +(43) CometFilter Input [2]: [s_county#27, s_state#28] Condition : (isnotnull(s_county#27) AND isnotnull(s_state#28)) -(42) ColumnarToRow [codegen id : 7] +(44) ColumnarToRow [codegen id : 7] Input [2]: [s_county#27, s_state#28] -(43) BroadcastExchange +(45) BroadcastExchange Input [2]: [s_county#27, s_state#28] Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [plan_id=6] -(44) BroadcastHashJoin [codegen id : 9] +(46) BroadcastHashJoin [codegen id : 9] Left keys [2]: [ca_county#25, ca_state#26] Right keys [2]: [s_county#27, s_state#28] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(47) Project [codegen id : 9] Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26, s_county#27, s_state#28] -(46) ReusedExchange [Reuses operator id: 65] +(48) ReusedExchange [Reuses operator id: 69] Output [1]: [d_date_sk#29] -(47) BroadcastHashJoin [codegen id : 9] +(49) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_sold_date_sk#22] Right keys [1]: [d_date_sk#29] Join type: Inner Join condition: None -(48) Project [codegen id : 9] +(50) Project [codegen id : 9] Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#29] -(49) HashAggregate [codegen id : 9] +(51) HashAggregate [codegen id : 9] Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] Keys [1]: [c_customer_sk#18] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] Aggregate Attributes [1]: [sum#30] Results [2]: [c_customer_sk#18, sum#31] -(50) Exchange +(52) RowToColumnar +Input [2]: [c_customer_sk#18, sum#31] + +(53) CometColumnarExchange Input [2]: [c_customer_sk#18, sum#31] -Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(51) HashAggregate [codegen id : 10] +(54) CometHashAggregate Input [2]: [c_customer_sk#18, sum#31] Keys [1]: [c_customer_sk#18] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#32] -Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#32,17,2) / 50) as int) AS segment#33] -(52) HashAggregate [codegen id : 10] -Input [1]: [segment#33] -Keys [1]: [segment#33] +(55) CometHashAggregate +Input [1]: [segment#32] +Keys [1]: [segment#32] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [2]: [segment#33, count#35] -(53) Exchange -Input [2]: [segment#33, count#35] -Arguments: hashpartitioning(segment#33, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(56) CometColumnarExchange +Input [2]: [segment#32, count#33] +Arguments: hashpartitioning(segment#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(54) HashAggregate [codegen id : 11] -Input [2]: [segment#33, count#35] -Keys [1]: [segment#33] +(57) CometHashAggregate +Input [2]: [segment#32, count#33] +Keys [1]: [segment#32] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [3]: [segment#33, count(1)#36 AS num_customers#37, (segment#33 * 50) AS segment_base#38] -(55) TakeOrderedAndProject -Input [3]: [segment#33, num_customers#37, segment_base#38] -Arguments: 100, [segment#33 ASC NULLS FIRST, num_customers#37 ASC NULLS FIRST], [segment#33, num_customers#37, segment_base#38] +(58) CometTakeOrderedAndProject +Input [3]: [segment#32, num_customers#34, segment_base#35] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#32 ASC NULLS FIRST,num_customers#34 ASC NULLS FIRST], output=[segment#32,num_customers#34,segment_base#35]), 100, [segment#32 ASC NULLS FIRST, num_customers#34 ASC NULLS FIRST], [segment#32, num_customers#34, segment_base#35] + +(59) ColumnarToRow [codegen id : 10] +Input [3]: [segment#32, num_customers#34, segment_base#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (60) -+- * ColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.date_dim (56) +BroadcastExchange (64) ++- * ColumnarToRow (63) + +- CometProject (62) + +- CometFilter (61) + +- CometScan parquet spark_catalog.default.date_dim (60) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#39, d_moy#40] +Output [3]: [d_date_sk#17, d_year#36, d_moy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter -Input [3]: [d_date_sk#17, d_year#39, d_moy#40] -Condition : ((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 = 12)) AND (d_year#39 = 1998)) AND isnotnull(d_date_sk#17)) +(61) CometFilter +Input [3]: [d_date_sk#17, d_year#36, d_moy#37] +Condition : ((((isnotnull(d_moy#37) AND isnotnull(d_year#36)) AND (d_moy#37 = 12)) AND (d_year#36 = 1998)) AND isnotnull(d_date_sk#17)) -(58) CometProject -Input [3]: [d_date_sk#17, d_year#39, d_moy#40] +(62) CometProject +Input [3]: [d_date_sk#17, d_year#36, d_moy#37] Arguments: [d_date_sk#17], [d_date_sk#17] -(59) ColumnarToRow [codegen id : 1] +(63) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] -(60) BroadcastExchange +(64) BroadcastExchange Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (65) -+- * ColumnarToRow (64) - +- CometProject (63) - +- CometFilter (62) - +- CometScan parquet spark_catalog.default.date_dim (61) +Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (69) ++- * ColumnarToRow (68) + +- CometProject (67) + +- CometFilter (66) + +- CometScan parquet spark_catalog.default.date_dim (65) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#29, d_month_seq#41] +Output [2]: [d_date_sk#29, d_month_seq#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(62) CometFilter -Input [2]: [d_date_sk#29, d_month_seq#41] -Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= Subquery scalar-subquery#42, [id=#43])) AND (d_month_seq#41 <= Subquery scalar-subquery#44, [id=#45])) AND isnotnull(d_date_sk#29)) +(66) CometFilter +Input [2]: [d_date_sk#29, d_month_seq#38] +Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#29)) -(63) CometProject -Input [2]: [d_date_sk#29, d_month_seq#41] +(67) CometProject +Input [2]: [d_date_sk#29, d_month_seq#38] Arguments: [d_date_sk#29], [d_date_sk#29] -(64) ColumnarToRow [codegen id : 1] +(68) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#29] -(65) BroadcastExchange +(69) BroadcastExchange Input [1]: [d_date_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:4 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* ColumnarToRow (72) -+- CometHashAggregate (71) - +- CometExchange (70) - +- CometHashAggregate (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan parquet spark_catalog.default.date_dim (66) +Subquery:4 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#39, [id=#40] +* ColumnarToRow (76) ++- CometHashAggregate (75) + +- CometColumnarExchange (74) + +- CometHashAggregate (73) + +- CometProject (72) + +- CometFilter (71) + +- CometScan parquet spark_catalog.default.date_dim (70) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#46, d_year#47, d_moy#48] +Output [3]: [d_month_seq#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(67) CometFilter -Input [3]: [d_month_seq#46, d_year#47, d_moy#48] -Condition : (((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 12)) +(71) CometFilter +Input [3]: [d_month_seq#43, d_year#44, d_moy#45] +Condition : (((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 1998)) AND (d_moy#45 = 12)) -(68) CometProject -Input [3]: [d_month_seq#46, d_year#47, d_moy#48] -Arguments: [(d_month_seq + 1)#49], [(d_month_seq#46 + 1) AS (d_month_seq + 1)#49] +(72) CometProject +Input [3]: [d_month_seq#43, d_year#44, d_moy#45] +Arguments: [(d_month_seq + 1)#46], [(d_month_seq#43 + 1) AS (d_month_seq + 1)#46] -(69) CometHashAggregate -Input [1]: [(d_month_seq + 1)#49] -Keys [1]: [(d_month_seq + 1)#49] +(73) CometHashAggregate +Input [1]: [(d_month_seq + 1)#46] +Keys [1]: [(d_month_seq + 1)#46] Functions: [] -(70) CometExchange -Input [1]: [(d_month_seq + 1)#49] -Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(74) CometColumnarExchange +Input [1]: [(d_month_seq + 1)#46] +Arguments: hashpartitioning((d_month_seq + 1)#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(71) CometHashAggregate -Input [1]: [(d_month_seq + 1)#49] -Keys [1]: [(d_month_seq + 1)#49] +(75) CometHashAggregate +Input [1]: [(d_month_seq + 1)#46] +Keys [1]: [(d_month_seq + 1)#46] Functions: [] -(72) ColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#49] +(76) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 1)#46] -Subquery:5 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* ColumnarToRow (79) -+- CometHashAggregate (78) - +- CometExchange (77) - +- CometHashAggregate (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +Subquery:5 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +* ColumnarToRow (83) ++- CometHashAggregate (82) + +- CometColumnarExchange (81) + +- CometHashAggregate (80) + +- CometProject (79) + +- CometFilter (78) + +- CometScan parquet spark_catalog.default.date_dim (77) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#50, d_year#51, d_moy#52] +Output [3]: [d_month_seq#47, d_year#48, d_moy#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(74) CometFilter -Input [3]: [d_month_seq#50, d_year#51, d_moy#52] -Condition : (((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 1998)) AND (d_moy#52 = 12)) +(78) CometFilter +Input [3]: [d_month_seq#47, d_year#48, d_moy#49] +Condition : (((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 1998)) AND (d_moy#49 = 12)) -(75) CometProject -Input [3]: [d_month_seq#50, d_year#51, d_moy#52] -Arguments: [(d_month_seq + 3)#53], [(d_month_seq#50 + 3) AS (d_month_seq + 3)#53] +(79) CometProject +Input [3]: [d_month_seq#47, d_year#48, d_moy#49] +Arguments: [(d_month_seq + 3)#50], [(d_month_seq#47 + 3) AS (d_month_seq + 3)#50] -(76) CometHashAggregate -Input [1]: [(d_month_seq + 3)#53] -Keys [1]: [(d_month_seq + 3)#53] +(80) CometHashAggregate +Input [1]: [(d_month_seq + 3)#50] +Keys [1]: [(d_month_seq + 3)#50] Functions: [] -(77) CometExchange -Input [1]: [(d_month_seq + 3)#53] -Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +(81) CometColumnarExchange +Input [1]: [(d_month_seq + 3)#50] +Arguments: hashpartitioning((d_month_seq + 3)#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(78) CometHashAggregate -Input [1]: [(d_month_seq + 3)#53] -Keys [1]: [(d_month_seq + 3)#53] +(82) CometHashAggregate +Input [1]: [(d_month_seq + 3)#50] +Keys [1]: [(d_month_seq + 3)#50] Functions: [] -(79) ColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#53] +(83) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 3)#50] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index 90604339a2..0d8b894b24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -1,13 +1,13 @@ -TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (11) - HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - InputAdapter - Exchange [segment] #1 - WholeStageCodegen (10) - HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - InputAdapter - Exchange [c_customer_sk] #2 +WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [segment,num_customers,segment_base] + CometHashAggregate [segment,count] + CometColumnarExchange [segment] #1 + CometHashAggregate [segment] + CometHashAggregate [c_customer_sk,sum] + CometColumnarExchange [c_customer_sk] #2 + RowToColumnar WholeStageCodegen (9) HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] Project [c_customer_sk,ss_ext_sales_price] @@ -18,52 +18,54 @@ TakeOrderedAndProject [segment,num_customers,segment_base] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [c_customer_sk,c_current_addr_sk] + ColumnarToRow InputAdapter - Exchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (4) - HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - BroadcastHashJoin [customer_sk,c_customer_sk] - Project [customer_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometUnion - CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [c_customer_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk] + BroadcastHashJoin [customer_sk,c_customer_sk] + Project [customer_sk] + BroadcastHashJoin [sold_date_sk,d_date_sk] + Project [sold_date_sk,customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_category,i_class,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) @@ -83,7 +85,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #9 + CometColumnarExchange [(d_month_seq + 1)] #9 CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] CometFilter [d_year,d_moy] @@ -93,7 +95,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #10 + CometColumnarExchange [(d_month_seq + 3)] #10 CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] CometFilter [d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt index 2efd757741..9dc4ba9896 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt @@ -1,25 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +* ColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometHashAggregate (21) + +- CometColumnarExchange (20) + +- RowToColumnar (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) (unknown) Scan parquet spark_catalog.default.date_dim @@ -108,18 +110,22 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [3]: [i_brand#9, i_brand_id#8, sum#12] -(19) Exchange +(19) RowToColumnar Input [3]: [i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) HashAggregate [codegen id : 4] +(20) CometColumnarExchange +Input [3]: [i_brand#9, i_brand_id#8, sum#12] +Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(21) CometHashAggregate Input [3]: [i_brand#9, i_brand_id#8, sum#12] Keys [2]: [i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [3]: [i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(21) TakeOrderedAndProject -Input [3]: [brand_id#14, brand#15, ext_price#16] -Arguments: 100, [ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [brand_id#14, brand#15, ext_price#16] +(22) CometTakeOrderedAndProject +Input [3]: [brand_id#13, brand#14, ext_price#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), 100, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] + +(23) ColumnarToRow [codegen id : 4] +Input [3]: [brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt index 7a0fe88633..368ca6f56a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt @@ -1,31 +1,33 @@ -TakeOrderedAndProject [ext_price,brand_id,brand] - WholeStageCodegen (4) - HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - InputAdapter - Exchange [i_brand,i_brand_id] #1 - WholeStageCodegen (3) - HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ext_price,brand_id,brand] + CometHashAggregate [i_brand,i_brand_id,sum] + CometColumnarExchange [i_brand,i_brand_id] #1 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 0fb6eb172f..15d5137015 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -1,67 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * ColumnarToRow (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * ColumnarToRow (46) - : : : +- CometFilter (45) - : : : +- CometScan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +* ColumnarToRow (67) ++- CometTakeOrderedAndProject (66) + +- CometHashAggregate (65) + +- CometColumnarExchange (64) + +- CometHashAggregate (63) + +- CometUnion (62) + :- CometHashAggregate (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- CometHashAggregate (45) + : +- CometColumnarExchange (44) + : +- RowToColumnar (43) + : +- * HashAggregate (42) + : +- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (32) + : : : : +- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) + : : : +- ReusedExchange (33) + : : +- ReusedExchange (36) + : +- ReusedExchange (39) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) + +- RowToColumnar (59) + +- * HashAggregate (58) + +- * Project (57) + +- * BroadcastHashJoin Inner BuildRight (56) + :- * Project (54) + : +- * BroadcastHashJoin Inner BuildRight (53) + : :- * Project (51) + : : +- * BroadcastHashJoin Inner BuildRight (50) + : : :- * ColumnarToRow (48) + : : : +- CometFilter (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- ReusedExchange (49) + : +- ReusedExchange (52) + +- ReusedExchange (55) (unknown) Scan parquet spark_catalog.default.store_sales @@ -79,7 +83,7 @@ Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 68] +(4) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -187,219 +191,224 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#13] Results [2]: [i_item_id#10, sum#14] -(27) Exchange +(27) RowToColumnar Input [2]: [i_item_id#10, sum#14] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) HashAggregate [codegen id : 6] +(28) CometColumnarExchange +Input [2]: [i_item_id#10, sum#14] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(29) CometHashAggregate Input [2]: [i_item_id#10, sum#14] Keys [1]: [i_item_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_item_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Output [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) +(31) CometFilter +Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] +Condition : (isnotnull(cs_bill_addr_sk#15) AND isnotnull(cs_item_sk#16)) -(31) ColumnarToRow [codegen id : 11] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +(32) ColumnarToRow [codegen id : 10] +Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -(32) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#22] +(33) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#20] -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] +(34) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(34) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] +(35) Project [codegen id : 10] +Output [3]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17] +Input [5]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#20] -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#23] +(36) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#21] -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#23] +(37) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_bill_addr_sk#15] +Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(37) Project [codegen id : 11] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] +(38) Project [codegen id : 10] +Output [2]: [cs_item_sk#16, cs_ext_sales_price#17] +Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, ca_address_sk#21] -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#24, i_item_id#25] +(39) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#22, i_item_id#23] -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#24] +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_item_sk#16] +Right keys [1]: [i_item_sk#22] Join type: Inner Join condition: None -(40) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#19, i_item_id#25] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] +(41) Project [codegen id : 10] +Output [2]: [cs_ext_sales_price#17, i_item_id#23] +Input [4]: [cs_item_sk#16, cs_ext_sales_price#17, i_item_sk#22, i_item_id#23] + +(42) HashAggregate [codegen id : 10] +Input [2]: [cs_ext_sales_price#17, i_item_id#23] +Keys [1]: [i_item_id#23] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#17))] +Aggregate Attributes [1]: [sum#24] +Results [2]: [i_item_id#23, sum#25] -(41) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#19, i_item_id#25] -Keys [1]: [i_item_id#25] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_item_id#25, sum#27] +(43) RowToColumnar +Input [2]: [i_item_id#23, sum#25] -(42) Exchange -Input [2]: [i_item_id#25, sum#27] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(44) CometColumnarExchange +Input [2]: [i_item_id#23, sum#25] +Arguments: hashpartitioning(i_item_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#25, sum#27] -Keys [1]: [i_item_id#25] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] -Results [2]: [i_item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] +(45) CometHashAggregate +Input [2]: [i_item_id#23, sum#25] +Keys [1]: [i_item_id#23] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#17))] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) +(47) CometFilter +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) -(46) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +(48) ColumnarToRow [codegen id : 15] +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -(47) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#35] +(49) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#31] -(48) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#35] +(50) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#29] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(49) Project [codegen id : 17] -Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] +(51) Project [codegen id : 15] +Output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] +Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] -(50) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#36] +(52) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#32] -(51) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#31] -Right keys [1]: [ca_address_sk#36] +(53) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_bill_addr_sk#27] +Right keys [1]: [ca_address_sk#32] Join type: Inner Join condition: None -(52) Project [codegen id : 17] -Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] +(54) Project [codegen id : 15] +Output [2]: [ws_item_sk#26, ws_ext_sales_price#28] +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] -(53) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#37, i_item_id#38] +(55) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#33, i_item_id#34] -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#30] -Right keys [1]: [i_item_sk#37] +(56) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#33] Join type: Inner Join condition: None -(55) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#32, i_item_id#38] -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] +(57) Project [codegen id : 15] +Output [2]: [ws_ext_sales_price#28, i_item_id#34] +Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_item_id#34] -(56) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#32, i_item_id#38] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum#39] -Results [2]: [i_item_id#38, sum#40] +(58) HashAggregate [codegen id : 15] +Input [2]: [ws_ext_sales_price#28, i_item_id#34] +Keys [1]: [i_item_id#34] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] +Aggregate Attributes [1]: [sum#35] +Results [2]: [i_item_id#34, sum#36] -(57) Exchange -Input [2]: [i_item_id#38, sum#40] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(59) RowToColumnar +Input [2]: [i_item_id#34, sum#36] -(58) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#38, sum#40] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] -Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] +(60) CometColumnarExchange +Input [2]: [i_item_id#34, sum#36] +Arguments: hashpartitioning(i_item_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(59) Union +(61) CometHashAggregate +Input [2]: [i_item_id#34, sum#36] +Keys [1]: [i_item_id#34] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -(60) HashAggregate [codegen id : 19] -Input [2]: [i_item_id#10, total_sales#16] +(62) CometUnion +Child 0 Input [2]: [i_item_id#10, total_sales#37] +Child 1 Input [2]: [i_item_id#23, total_sales#38] +Child 2 Input [2]: [i_item_id#34, total_sales#39] + +(63) CometHashAggregate +Input [2]: [i_item_id#10, total_sales#37] Keys [1]: [i_item_id#10] -Functions [1]: [partial_sum(total_sales#16)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [3]: [i_item_id#10, sum#45, isEmpty#46] +Functions [1]: [partial_sum(total_sales#37)] -(61) Exchange -Input [3]: [i_item_id#10, sum#45, isEmpty#46] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(64) CometColumnarExchange +Input [3]: [i_item_id#10, sum#40, isEmpty#41] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(62) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#10, sum#45, isEmpty#46] +(65) CometHashAggregate +Input [3]: [i_item_id#10, sum#40, isEmpty#41] Keys [1]: [i_item_id#10] -Functions [1]: [sum(total_sales#16)] -Aggregate Attributes [1]: [sum(total_sales#16)#47] -Results [2]: [i_item_id#10, sum(total_sales#16)#47 AS total_sales#48] +Functions [1]: [sum(total_sales#37)] + +(66) CometTakeOrderedAndProject +Input [2]: [i_item_id#10, total_sales#42] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#42 ASC NULLS FIRST], output=[i_item_id#10,total_sales#42]), 100, [total_sales#42 ASC NULLS FIRST], [i_item_id#10, total_sales#42] -(63) TakeOrderedAndProject -Input [2]: [i_item_id#10, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#10, total_sales#48] +(67) ColumnarToRow [codegen id : 16] +Input [2]: [i_item_id#10, total_sales#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#49, d_moy#50] +Output [3]: [d_date_sk#6, d_year#43, d_moy#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(65) CometFilter -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2001)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#6)) +(69) CometFilter +Input [3]: [d_date_sk#6, d_year#43, d_moy#44] +Condition : ((((isnotnull(d_year#43) AND isnotnull(d_moy#44)) AND (d_year#43 = 2001)) AND (d_moy#44 = 2)) AND isnotnull(d_date_sk#6)) -(66) CometProject -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +(70) CometProject +Input [3]: [d_date_sk#6, d_year#43, d_moy#44] Arguments: [d_date_sk#6], [d_date_sk#6] -(67) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index f781ed1f7b..68bd217820 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -1,105 +1,101 @@ -TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (20) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sales,i_item_id] + CometHashAggregate [i_item_id,sum,isEmpty] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,total_sales] + CometUnion + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #6 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_item_id] - CometFilter [i_color] - CometScan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [i_item_id] + CometFilter [i_color] + CometScan parquet spark_catalog.default.item [i_item_id,i_color] + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #7 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [ca_address_sk] #4 + ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #8 + RowToColumnar + WholeStageCodegen (15) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [ca_address_sk] #4 + ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index 8302389585..70e5ea9d45 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * Filter (25) - : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) - +- BroadcastExchange (42) - +- * Project (41) - +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) +TakeOrderedAndProject (47) ++- * Project (46) + +- * BroadcastHashJoin Inner BuildRight (45) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * ColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- CometHashAggregate (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.call_center (13) + : +- BroadcastExchange (36) + : +- * Project (35) + : +- Window (34) + : +- * ColumnarToRow (33) + : +- CometSort (32) + : +- ReusedExchange (31) + +- BroadcastExchange (44) + +- * Project (43) + +- Window (42) + +- * ColumnarToRow (41) + +- CometSort (40) + +- ReusedExchange (39) (unknown) Scan parquet spark_catalog.default.item @@ -89,7 +91,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(10) ReusedExchange [Reuses operator id: 51] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -137,125 +139,124 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#6))] Aggregate Attributes [1]: [sum#14] Results [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -(20) Exchange +(20) RowToColumnar Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) HashAggregate [codegen id : 5] +(21) CometColumnarExchange +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) CometHashAggregate Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#16] -Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS _w0#18] -(22) Exchange -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(23) CometColumnarExchange +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(23) Sort [codegen id : 6] -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 +(24) CometSort +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(24) Window -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(25) ColumnarToRow [codegen id : 5] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] + +(26) Window +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#18], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +(27) Filter [codegen id : 6] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(26) Window -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] - -(27) Filter [codegen id : 22] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(28) Project [codegen id : 22] -Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(29) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] - -(30) HashAggregate [codegen id : 12] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] -Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] - -(31) Exchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(32) Sort [codegen id : 13] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 - -(33) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) Project [codegen id : 14] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] - -(35) BroadcastExchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] - -(36) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] +(28) Window +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] +Arguments: [avg(_w0#17) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#19], [i_category#3, i_brand#2, cc_name#13, d_year#10] + +(29) Filter [codegen id : 19] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] +Condition : ((isnotnull(avg_monthly_sales#19) AND (avg_monthly_sales#19 > 0.000000)) AND CASE WHEN (avg_monthly_sales#19 > 0.000000) THEN ((abs((sum_sales#16 - avg_monthly_sales#19)) / avg_monthly_sales#19) > 0.1000000000000000) END) + +(30) Project [codegen id : 19] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] + +(31) ReusedExchange [Reuses operator id: 23] +Output [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] + +(32) CometSort +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16], [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, cc_name#22 ASC NULLS FIRST, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] + +(33) ColumnarToRow [codegen id : 11] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] + +(34) Window +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [rank(d_year#23, d_moy#24) windowspecdefinition(i_category#20, i_brand#21, cc_name#22, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#25], [i_category#20, i_brand#21, cc_name#22], [d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] + +(35) Project [codegen id : 12] +Output [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#16 AS sum_sales#26, rn#25] +Input [7]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16, rn#25] + +(36) BroadcastExchange +Input [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#26, rn#25] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 19] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] +Right keys [4]: [i_category#20, i_brand#21, cc_name#22, (rn#25 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +(38) Project [codegen id : 19] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#26] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, cc_name#22, sum_sales#26, rn#25] + +(39) ReusedExchange [Reuses operator id: 23] +Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] -(38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +(40) CometSort +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] +Arguments: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16], [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] -(39) Sort [codegen id : 20] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 +(41) ColumnarToRow [codegen id : 17] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] -(40) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +(42) Window +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] +Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] -(41) Project [codegen id : 21] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] +(43) Project [codegen id : 18] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#16 AS sum_sales#33, rn#32] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16, rn#32] -(42) BroadcastExchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] +(44) BroadcastExchange +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#33, rn#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] +(45) BroadcastHashJoin [codegen id : 19] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#32 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +(46) Project [codegen id : 19] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#26 AS psum#34, sum_sales#33 AS nsum#35] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#26, i_category#27, i_brand#28, cc_name#29, sum_sales#33, rn#32] -(45) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +(47) TakeOrderedAndProject +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#34, nsum#35] +Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#34, nsum#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim @@ -265,15 +266,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(49) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(51) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index 3bc01343ae..b9bd07df45 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (19) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,18 +8,18 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (6) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometSort [i_category,i_brand,cc_name,d_year,d_moy] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + RowToColumnar WholeStageCodegen (4) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] @@ -57,25 +57,23 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (11) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + CometSort [i_category,i_brand,cc_name,d_year,d_moy] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #7 + WholeStageCodegen (18) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (17) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + CometSort [i_category,i_brand,cc_name,d_year,d_moy] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index e3b68cabe2..d31d811d90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -1,53 +1,59 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Filter (16) - : : +- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (10) - : +- BroadcastExchange (30) - : +- * Filter (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometFilter (18) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- BroadcastExchange (46) - +- * Filter (45) - +- * HashAggregate (44) - +- Exchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * ColumnarToRow (35) - : : +- CometFilter (34) - : : +- CometScan parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (36) - +- ReusedExchange (39) +TakeOrderedAndProject (55) ++- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * ColumnarToRow (18) + : : +- CometFilter (17) + : : +- CometHashAggregate (16) + : : +- CometColumnarExchange (15) + : : +- RowToColumnar (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- ReusedExchange (10) + : +- BroadcastExchange (34) + : +- * ColumnarToRow (33) + : +- CometFilter (32) + : +- CometHashAggregate (31) + : +- CometColumnarExchange (30) + : +- RowToColumnar (29) + : +- * HashAggregate (28) + : +- * Project (27) + : +- * BroadcastHashJoin Inner BuildRight (26) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * ColumnarToRow (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (19) + : : +- ReusedExchange (22) + : +- ReusedExchange (25) + +- BroadcastExchange (52) + +- * ColumnarToRow (51) + +- CometFilter (50) + +- CometHashAggregate (49) + +- CometColumnarExchange (48) + +- RowToColumnar (47) + +- * HashAggregate (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * ColumnarToRow (39) + : : +- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (40) + +- ReusedExchange (43) (unknown) Scan parquet spark_catalog.default.store_sales @@ -93,7 +99,7 @@ Join condition: None Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6] -(10) ReusedExchange [Reuses operator id: 60] +(10) ReusedExchange [Reuses operator id: 66] Output [1]: [d_date_sk#7] (11) BroadcastHashJoin [codegen id : 4] @@ -113,272 +119,284 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#8] Results [2]: [i_item_id#6, sum#9] -(14) Exchange +(14) RowToColumnar Input [2]: [i_item_id#6, sum#9] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 15] +(15) CometColumnarExchange +Input [2]: [i_item_id#6, sum#9] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [2]: [i_item_id#6, sum#9] Keys [1]: [i_item_id#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#10] -Results [2]: [i_item_id#6 AS item_id#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#10,17,2) AS ss_item_rev#12] -(16) Filter [codegen id : 15] -Input [2]: [item_id#11, ss_item_rev#12] -Condition : isnotnull(ss_item_rev#12) +(17) CometFilter +Input [2]: [item_id#10, ss_item_rev#11] +Condition : isnotnull(ss_item_rev#11) + +(18) ColumnarToRow [codegen id : 15] +Input [2]: [item_id#10, ss_item_rev#11] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +Output [3]: [cs_item_sk#12, cs_ext_sales_price#13, cs_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(18) CometFilter -Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] -Condition : isnotnull(cs_item_sk#13) +(20) CometFilter +Input [3]: [cs_item_sk#12, cs_ext_sales_price#13, cs_sold_date_sk#14] +Condition : isnotnull(cs_item_sk#12) -(19) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +(21) ColumnarToRow [codegen id : 8] +Input [3]: [cs_item_sk#12, cs_ext_sales_price#13, cs_sold_date_sk#14] -(20) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#17, i_item_id#18] +(22) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#16, i_item_id#17] -(21) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#13] -Right keys [1]: [i_item_sk#17] +(23) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#12] +Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(22) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18] -Input [5]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_sk#17, i_item_id#18] +(24) Project [codegen id : 8] +Output [3]: [cs_ext_sales_price#13, cs_sold_date_sk#14, i_item_id#17] +Input [5]: [cs_item_sk#12, cs_ext_sales_price#13, cs_sold_date_sk#14, i_item_sk#16, i_item_id#17] -(23) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#19] +(25) ReusedExchange [Reuses operator id: 66] +Output [1]: [d_date_sk#18] -(24) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#15] -Right keys [1]: [d_date_sk#19] +(26) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#14] +Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(25) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#14, i_item_id#18] -Input [4]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18, d_date_sk#19] - -(26) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#14, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#14))] -Aggregate Attributes [1]: [sum#20] -Results [2]: [i_item_id#18, sum#21] - -(27) Exchange -Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(28) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#18, sum#21] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#14))#22] -Results [2]: [i_item_id#18 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#14))#22,17,2) AS cs_item_rev#24] - -(29) Filter [codegen id : 9] -Input [2]: [item_id#23, cs_item_rev#24] -Condition : isnotnull(cs_item_rev#24) - -(30) BroadcastExchange -Input [2]: [item_id#23, cs_item_rev#24] +(27) Project [codegen id : 8] +Output [2]: [cs_ext_sales_price#13, i_item_id#17] +Input [4]: [cs_ext_sales_price#13, cs_sold_date_sk#14, i_item_id#17, d_date_sk#18] + +(28) HashAggregate [codegen id : 8] +Input [2]: [cs_ext_sales_price#13, i_item_id#17] +Keys [1]: [i_item_id#17] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#13))] +Aggregate Attributes [1]: [sum#19] +Results [2]: [i_item_id#17, sum#20] + +(29) RowToColumnar +Input [2]: [i_item_id#17, sum#20] + +(30) CometColumnarExchange +Input [2]: [i_item_id#17, sum#20] +Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(31) CometHashAggregate +Input [2]: [i_item_id#17, sum#20] +Keys [1]: [i_item_id#17] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#13))] + +(32) CometFilter +Input [2]: [item_id#21, cs_item_rev#22] +Condition : isnotnull(cs_item_rev#22) + +(33) ColumnarToRow [codegen id : 9] +Input [2]: [item_id#21, cs_item_rev#22] + +(34) BroadcastExchange +Input [2]: [item_id#21, cs_item_rev#22] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#11] -Right keys [1]: [item_id#23] +(35) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [item_id#10] +Right keys [1]: [item_id#21] Join type: Inner -Join condition: ((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) +Join condition: ((((cast(ss_item_rev#11 as decimal(19,3)) >= (0.9 * cs_item_rev#22)) AND (cast(ss_item_rev#11 as decimal(20,3)) <= (1.1 * cs_item_rev#22))) AND (cast(cs_item_rev#22 as decimal(19,3)) >= (0.9 * ss_item_rev#11))) AND (cast(cs_item_rev#22 as decimal(20,3)) <= (1.1 * ss_item_rev#11))) -(32) Project [codegen id : 15] -Output [3]: [item_id#11, ss_item_rev#12, cs_item_rev#24] -Input [4]: [item_id#11, ss_item_rev#12, item_id#23, cs_item_rev#24] +(36) Project [codegen id : 15] +Output [3]: [item_id#10, ss_item_rev#11, cs_item_rev#22] +Input [4]: [item_id#10, ss_item_rev#11, item_id#21, cs_item_rev#22] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] +Output [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ws_sold_date_sk#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#25) +(38) CometFilter +Input [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] +Condition : isnotnull(ws_item_sk#23) -(35) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] +(39) ColumnarToRow [codegen id : 13] +Input [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] -(36) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#29, i_item_id#30] +(40) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#27, i_item_id#28] -(37) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#25] -Right keys [1]: [i_item_sk#29] +(41) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(38) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] -Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#29, i_item_id#30] +(42) Project [codegen id : 13] +Output [3]: [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28] +Input [5]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_sk#27, i_item_id#28] -(39) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#31] +(43) ReusedExchange [Reuses operator id: 66] +Output [1]: [d_date_sk#29] -(40) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#31] +(44) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_sold_date_sk#25] +Right keys [1]: [d_date_sk#29] Join type: Inner Join condition: None -(41) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#26, i_item_id#30] -Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30, d_date_sk#31] - -(42) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#26, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#30, sum#33] - -(43) Exchange -Input [2]: [i_item_id#30, sum#33] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(44) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#30, sum#33] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#34] -Results [2]: [i_item_id#30 AS item_id#35, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#34,17,2) AS ws_item_rev#36] - -(45) Filter [codegen id : 14] -Input [2]: [item_id#35, ws_item_rev#36] -Condition : isnotnull(ws_item_rev#36) - -(46) BroadcastExchange -Input [2]: [item_id#35, ws_item_rev#36] +(45) Project [codegen id : 13] +Output [2]: [ws_ext_sales_price#24, i_item_id#28] +Input [4]: [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28, d_date_sk#29] + +(46) HashAggregate [codegen id : 13] +Input [2]: [ws_ext_sales_price#24, i_item_id#28] +Keys [1]: [i_item_id#28] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#24))] +Aggregate Attributes [1]: [sum#30] +Results [2]: [i_item_id#28, sum#31] + +(47) RowToColumnar +Input [2]: [i_item_id#28, sum#31] + +(48) CometColumnarExchange +Input [2]: [i_item_id#28, sum#31] +Arguments: hashpartitioning(i_item_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(49) CometHashAggregate +Input [2]: [i_item_id#28, sum#31] +Keys [1]: [i_item_id#28] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#24))] + +(50) CometFilter +Input [2]: [item_id#32, ws_item_rev#33] +Condition : isnotnull(ws_item_rev#33) + +(51) ColumnarToRow [codegen id : 14] +Input [2]: [item_id#32, ws_item_rev#33] + +(52) BroadcastExchange +Input [2]: [item_id#32, ws_item_rev#33] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#11] -Right keys [1]: [item_id#35] +(53) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [item_id#10] +Right keys [1]: [item_id#32] Join type: Inner -Join condition: ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * ws_item_rev#36)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) +Join condition: ((((((((cast(ss_item_rev#11 as decimal(19,3)) >= (0.9 * ws_item_rev#33)) AND (cast(ss_item_rev#11 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(cs_item_rev#22 as decimal(19,3)) >= (0.9 * ws_item_rev#33))) AND (cast(cs_item_rev#22 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * ss_item_rev#11))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * ss_item_rev#11))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * cs_item_rev#22))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * cs_item_rev#22))) -(48) Project [codegen id : 15] -Output [8]: [item_id#11, ss_item_rev#12, (((ss_item_rev#12 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ss_dev#37, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS cs_dev#38, ws_item_rev#36, (((ws_item_rev#36 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ws_dev#39, (((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36) / 3) AS average#40] -Input [5]: [item_id#11, ss_item_rev#12, cs_item_rev#24, item_id#35, ws_item_rev#36] +(54) Project [codegen id : 15] +Output [8]: [item_id#10, ss_item_rev#11, (((ss_item_rev#11 / ((ss_item_rev#11 + cs_item_rev#22) + ws_item_rev#33)) / 3) * 100) AS ss_dev#34, cs_item_rev#22, (((cs_item_rev#22 / ((ss_item_rev#11 + cs_item_rev#22) + ws_item_rev#33)) / 3) * 100) AS cs_dev#35, ws_item_rev#33, (((ws_item_rev#33 / ((ss_item_rev#11 + cs_item_rev#22) + ws_item_rev#33)) / 3) * 100) AS ws_dev#36, (((ss_item_rev#11 + cs_item_rev#22) + ws_item_rev#33) / 3) AS average#37] +Input [5]: [item_id#10, ss_item_rev#11, cs_item_rev#22, item_id#32, ws_item_rev#33] -(49) TakeOrderedAndProject -Input [8]: [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] -Arguments: 100, [item_id#11 ASC NULLS FIRST, ss_item_rev#12 ASC NULLS FIRST], [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] +(55) TakeOrderedAndProject +Input [8]: [item_id#10, ss_item_rev#11, ss_dev#34, cs_item_rev#22, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] +Arguments: 100, [item_id#10 ASC NULLS FIRST, ss_item_rev#11 ASC NULLS FIRST], [item_id#10, ss_item_rev#11, ss_dev#34, cs_item_rev#22, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (60) -+- * Project (59) - +- * BroadcastHashJoin LeftSemi BuildRight (58) - :- * ColumnarToRow (52) - : +- CometFilter (51) - : +- CometScan parquet spark_catalog.default.date_dim (50) - +- BroadcastExchange (57) - +- * ColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan parquet spark_catalog.default.date_dim (53) +BroadcastExchange (66) ++- * Project (65) + +- * BroadcastHashJoin LeftSemi BuildRight (64) + :- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.date_dim (56) + +- BroadcastExchange (63) + +- * ColumnarToRow (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.date_dim (59) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_date#41] +Output [2]: [d_date_sk#7, d_date#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter -Input [2]: [d_date_sk#7, d_date#41] +(57) CometFilter +Input [2]: [d_date_sk#7, d_date#38] Condition : isnotnull(d_date_sk#7) -(52) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#7, d_date#41] +(58) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#7, d_date#38] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#42, d_week_seq#43] +Output [2]: [d_date#39, d_week_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(54) CometFilter -Input [2]: [d_date#42, d_week_seq#43] -Condition : (isnotnull(d_week_seq#43) AND (d_week_seq#43 = Subquery scalar-subquery#44, [id=#45])) +(60) CometFilter +Input [2]: [d_date#39, d_week_seq#40] +Condition : (isnotnull(d_week_seq#40) AND (d_week_seq#40 = Subquery scalar-subquery#41, [id=#42])) -(55) CometProject -Input [2]: [d_date#42, d_week_seq#43] -Arguments: [d_date#42], [d_date#42] +(61) CometProject +Input [2]: [d_date#39, d_week_seq#40] +Arguments: [d_date#39], [d_date#39] -(56) ColumnarToRow [codegen id : 1] -Input [1]: [d_date#42] +(62) ColumnarToRow [codegen id : 1] +Input [1]: [d_date#39] -(57) BroadcastExchange -Input [1]: [d_date#42] +(63) BroadcastExchange +Input [1]: [d_date#39] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=7] -(58) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_date#41] -Right keys [1]: [d_date#42] +(64) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [d_date#38] +Right keys [1]: [d_date#39] Join type: LeftSemi Join condition: None -(59) Project [codegen id : 2] +(65) Project [codegen id : 2] Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#41] +Input [2]: [d_date_sk#7, d_date#38] -(60) BroadcastExchange +(66) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 54 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* ColumnarToRow (64) -+- CometProject (63) - +- CometFilter (62) - +- CometScan parquet spark_catalog.default.date_dim (61) +Subquery:2 Hosting operator id = 60 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +* ColumnarToRow (70) ++- CometProject (69) + +- CometFilter (68) + +- CometScan parquet spark_catalog.default.date_dim (67) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#46, d_week_seq#47] +Output [2]: [d_date#43, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(62) CometFilter -Input [2]: [d_date#46, d_week_seq#47] -Condition : (isnotnull(d_date#46) AND (d_date#46 = 2000-01-03)) +(68) CometFilter +Input [2]: [d_date#43, d_week_seq#44] +Condition : (isnotnull(d_date#43) AND (d_date#43 = 2000-01-03)) -(63) CometProject -Input [2]: [d_date#46, d_week_seq#47] -Arguments: [d_week_seq#47], [d_week_seq#47] +(69) CometProject +Input [2]: [d_date#43, d_week_seq#44] +Arguments: [d_week_seq#44], [d_week_seq#44] -(64) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#47] +(70) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#44] -Subquery:3 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#25 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index d3bb0dc388..63eb226bb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -4,94 +4,100 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] Project [item_id,ss_item_rev,cs_item_rev] BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (2) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - ColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date] - CometFilter [d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + ColumnarToRow + InputAdapter + CometFilter [ss_item_rev] + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (2) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + ColumnarToRow + InputAdapter + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date] + CometFilter [d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_date] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - InputAdapter - Exchange [i_item_id] #6 - WholeStageCodegen (8) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_rev] + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #6 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (14) + ColumnarToRow + InputAdapter + CometFilter [ws_item_rev] + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #8 + RowToColumnar + WholeStageCodegen (13) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (14) - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (13) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt index 62b3f58686..d2241863e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt @@ -1,48 +1,51 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * HashAggregate (12) - : : : +- Exchange (11) - : : : +- * HashAggregate (10) - : : : +- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.date_dim (4) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.date_dim (19) - +- BroadcastExchange (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * HashAggregate (27) - : : +- ReusedExchange (26) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometFilter (29) - : +- CometScan parquet spark_catalog.default.store (28) - +- BroadcastExchange (38) - +- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +TakeOrderedAndProject (47) ++- * Project (46) + +- * BroadcastHashJoin Inner BuildRight (45) + :- * Project (27) + : +- * BroadcastHashJoin Inner BuildRight (26) + : :- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * ColumnarToRow (14) + : : : +- CometHashAggregate (13) + : : : +- CometColumnarExchange (12) + : : : +- RowToColumnar (11) + : : : +- * HashAggregate (10) + : : : +- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : +- BroadcastExchange (18) + : : +- * ColumnarToRow (17) + : : +- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.store (15) + : +- BroadcastExchange (25) + : +- * ColumnarToRow (24) + : +- CometProject (23) + : +- CometFilter (22) + : +- CometScan parquet spark_catalog.default.date_dim (21) + +- BroadcastExchange (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * ColumnarToRow (30) + : : +- CometHashAggregate (29) + : : +- ReusedExchange (28) + : +- BroadcastExchange (34) + : +- * ColumnarToRow (33) + : +- CometFilter (32) + : +- CometScan parquet spark_catalog.default.store (31) + +- BroadcastExchange (41) + +- * ColumnarToRow (40) + +- CometProject (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.date_dim (37) (unknown) Scan parquet spark_catalog.default.store_sales @@ -95,162 +98,167 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) T Aggregate Attributes [7]: [sum#7, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13] Results [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] -(11) Exchange +(11) RowToColumnar Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(12) HashAggregate [codegen id : 10] +(12) CometColumnarExchange +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] +Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(13) CometHashAggregate Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27] -Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21,17,2) AS sun_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22,17,2) AS mon_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23,17,2) AS tue_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24,17,2) AS wed_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25,17,2) AS thu_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26,17,2) AS fri_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27,17,2) AS sat_sales#34] + +(14) ColumnarToRow [codegen id : 10] +Input [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27] (unknown) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] +Output [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(14) CometFilter -Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] -Condition : (isnotnull(s_store_sk#35) AND isnotnull(s_store_id#36)) +(16) CometFilter +Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] +Condition : (isnotnull(s_store_sk#28) AND isnotnull(s_store_id#29)) -(15) ColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] +(17) ColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] -(16) BroadcastExchange -Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] +(18) BroadcastExchange +Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(17) BroadcastHashJoin [codegen id : 10] +(19) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#35] +Right keys [1]: [s_store_sk#28] Join type: Inner Join condition: None -(18) Project [codegen id : 10] -Output [10]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37] -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_sk#35, s_store_id#36, s_store_name#37] +(20) Project [codegen id : 10] +Output [10]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#29, s_store_name#30] +Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_sk#28, s_store_id#29, s_store_name#30] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#38, d_week_seq#39] +Output [2]: [d_month_seq#31, d_week_seq#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] ReadSchema: struct -(20) CometFilter -Input [2]: [d_month_seq#38, d_week_seq#39] -Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= 1212)) AND (d_month_seq#38 <= 1223)) AND isnotnull(d_week_seq#39)) +(22) CometFilter +Input [2]: [d_month_seq#31, d_week_seq#32] +Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1212)) AND (d_month_seq#31 <= 1223)) AND isnotnull(d_week_seq#32)) -(21) CometProject -Input [2]: [d_month_seq#38, d_week_seq#39] -Arguments: [d_week_seq#39], [d_week_seq#39] +(23) CometProject +Input [2]: [d_month_seq#31, d_week_seq#32] +Arguments: [d_week_seq#32], [d_week_seq#32] -(22) ColumnarToRow [codegen id : 4] -Input [1]: [d_week_seq#39] +(24) ColumnarToRow [codegen id : 4] +Input [1]: [d_week_seq#32] -(23) BroadcastExchange -Input [1]: [d_week_seq#39] +(25) BroadcastExchange +Input [1]: [d_week_seq#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(24) BroadcastHashJoin [codegen id : 10] +(26) BroadcastHashJoin [codegen id : 10] Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#39] +Right keys [1]: [d_week_seq#32] Join type: Inner Join condition: None -(25) Project [codegen id : 10] -Output [10]: [s_store_name#37 AS s_store_name1#40, d_week_seq#5 AS d_week_seq1#41, s_store_id#36 AS s_store_id1#42, sun_sales#28 AS sun_sales1#43, mon_sales#29 AS mon_sales1#44, tue_sales#30 AS tue_sales1#45, wed_sales#31 AS wed_sales1#46, thu_sales#32 AS thu_sales1#47, fri_sales#33 AS fri_sales1#48, sat_sales#34 AS sat_sales1#49] -Input [11]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37, d_week_seq#39] +(27) Project [codegen id : 10] +Output [10]: [s_store_name#30 AS s_store_name1#33, d_week_seq#5 AS d_week_seq1#34, s_store_id#29 AS s_store_id1#35, sun_sales#21 AS sun_sales1#36, mon_sales#22 AS mon_sales1#37, tue_sales#23 AS tue_sales1#38, wed_sales#24 AS wed_sales1#39, thu_sales#25 AS thu_sales1#40, fri_sales#26 AS fri_sales1#41, sat_sales#27 AS sat_sales1#42] +Input [11]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#29, s_store_name#30, d_week_seq#32] -(26) ReusedExchange [Reuses operator id: 11] -Output [9]: [d_week_seq#5, ss_store_sk#1, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] +(28) ReusedExchange [Reuses operator id: 12] +Output [9]: [d_week_seq#5, ss_store_sk#1, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] -(27) HashAggregate [codegen id : 9] -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] +(29) CometHashAggregate +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27] -Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21,17,2) AS sun_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22,17,2) AS mon_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23,17,2) AS tue_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24,17,2) AS wed_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25,17,2) AS thu_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26,17,2) AS fri_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27,17,2) AS sat_sales#34] + +(30) ColumnarToRow [codegen id : 9] +Input [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27] (unknown) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#57, s_store_id#58] +Output [2]: [s_store_sk#50, s_store_id#51] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(29) CometFilter -Input [2]: [s_store_sk#57, s_store_id#58] -Condition : (isnotnull(s_store_sk#57) AND isnotnull(s_store_id#58)) +(32) CometFilter +Input [2]: [s_store_sk#50, s_store_id#51] +Condition : (isnotnull(s_store_sk#50) AND isnotnull(s_store_id#51)) -(30) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#57, s_store_id#58] +(33) ColumnarToRow [codegen id : 7] +Input [2]: [s_store_sk#50, s_store_id#51] -(31) BroadcastExchange -Input [2]: [s_store_sk#57, s_store_id#58] +(34) BroadcastExchange +Input [2]: [s_store_sk#50, s_store_id#51] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#57] +Right keys [1]: [s_store_sk#50] Join type: Inner Join condition: None -(33) Project [codegen id : 9] -Output [9]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#58] -Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_sk#57, s_store_id#58] +(36) Project [codegen id : 9] +Output [9]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#51] +Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_sk#50, s_store_id#51] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#59, d_week_seq#60] +Output [2]: [d_month_seq#52, d_week_seq#53] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] ReadSchema: struct -(35) CometFilter -Input [2]: [d_month_seq#59, d_week_seq#60] -Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1224)) AND (d_month_seq#59 <= 1235)) AND isnotnull(d_week_seq#60)) +(38) CometFilter +Input [2]: [d_month_seq#52, d_week_seq#53] +Condition : (((isnotnull(d_month_seq#52) AND (d_month_seq#52 >= 1224)) AND (d_month_seq#52 <= 1235)) AND isnotnull(d_week_seq#53)) -(36) CometProject -Input [2]: [d_month_seq#59, d_week_seq#60] -Arguments: [d_week_seq#60], [d_week_seq#60] +(39) CometProject +Input [2]: [d_month_seq#52, d_week_seq#53] +Arguments: [d_week_seq#53], [d_week_seq#53] -(37) ColumnarToRow [codegen id : 8] -Input [1]: [d_week_seq#60] +(40) ColumnarToRow [codegen id : 8] +Input [1]: [d_week_seq#53] -(38) BroadcastExchange -Input [1]: [d_week_seq#60] +(41) BroadcastExchange +Input [1]: [d_week_seq#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#60] +Right keys [1]: [d_week_seq#53] Join type: Inner Join condition: None -(40) Project [codegen id : 9] -Output [9]: [d_week_seq#5 AS d_week_seq2#61, s_store_id#58 AS s_store_id2#62, sun_sales#28 AS sun_sales2#63, mon_sales#29 AS mon_sales2#64, tue_sales#30 AS tue_sales2#65, wed_sales#31 AS wed_sales2#66, thu_sales#32 AS thu_sales2#67, fri_sales#33 AS fri_sales2#68, sat_sales#34 AS sat_sales2#69] -Input [10]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#58, d_week_seq#60] +(43) Project [codegen id : 9] +Output [9]: [d_week_seq#5 AS d_week_seq2#54, s_store_id#51 AS s_store_id2#55, sun_sales#21 AS sun_sales2#56, mon_sales#22 AS mon_sales2#57, tue_sales#23 AS tue_sales2#58, wed_sales#24 AS wed_sales2#59, thu_sales#25 AS thu_sales2#60, fri_sales#26 AS fri_sales2#61, sat_sales#27 AS sat_sales2#62] +Input [10]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#51, d_week_seq#53] -(41) BroadcastExchange -Input [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] +(44) BroadcastExchange +Input [9]: [d_week_seq2#54, s_store_id2#55, sun_sales2#56, mon_sales2#57, tue_sales2#58, wed_sales2#59, thu_sales2#60, fri_sales2#61, sat_sales2#62] Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=7] -(42) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [s_store_id1#42, d_week_seq1#41] -Right keys [2]: [s_store_id2#62, (d_week_seq2#61 - 52)] +(45) BroadcastHashJoin [codegen id : 10] +Left keys [2]: [s_store_id1#35, d_week_seq1#34] +Right keys [2]: [s_store_id2#55, (d_week_seq2#54 - 52)] Join type: Inner Join condition: None -(43) Project [codegen id : 10] -Output [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1#43 / sun_sales2#63) AS (sun_sales1 / sun_sales2)#70, (mon_sales1#44 / mon_sales2#64) AS (mon_sales1 / mon_sales2)#71, (tue_sales1#45 / tue_sales2#65) AS (tue_sales1 / tue_sales2)#72, (wed_sales1#46 / wed_sales2#66) AS (wed_sales1 / wed_sales2)#73, (thu_sales1#47 / thu_sales2#67) AS (thu_sales1 / thu_sales2)#74, (fri_sales1#48 / fri_sales2#68) AS (fri_sales1 / fri_sales2)#75, (sat_sales1#49 / sat_sales2#69) AS (sat_sales1 / sat_sales2)#76] -Input [19]: [s_store_name1#40, d_week_seq1#41, s_store_id1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] +(46) Project [codegen id : 10] +Output [10]: [s_store_name1#33, s_store_id1#35, d_week_seq1#34, (sun_sales1#36 / sun_sales2#56) AS (sun_sales1 / sun_sales2)#63, (mon_sales1#37 / mon_sales2#57) AS (mon_sales1 / mon_sales2)#64, (tue_sales1#38 / tue_sales2#58) AS (tue_sales1 / tue_sales2)#65, (wed_sales1#39 / wed_sales2#59) AS (wed_sales1 / wed_sales2)#66, (thu_sales1#40 / thu_sales2#60) AS (thu_sales1 / thu_sales2)#67, (fri_sales1#41 / fri_sales2#61) AS (fri_sales1 / fri_sales2)#68, (sat_sales1#42 / sat_sales2#62) AS (sat_sales1 / sat_sales2)#69] +Input [19]: [s_store_name1#33, d_week_seq1#34, s_store_id1#35, sun_sales1#36, mon_sales1#37, tue_sales1#38, wed_sales1#39, thu_sales1#40, fri_sales1#41, sat_sales1#42, d_week_seq2#54, s_store_id2#55, sun_sales2#56, mon_sales2#57, tue_sales2#58, wed_sales2#59, thu_sales2#60, fri_sales2#61, sat_sales2#62] -(44) TakeOrderedAndProject -Input [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] -Arguments: 100, [s_store_name1#40 ASC NULLS FIRST, s_store_id1#42 ASC NULLS FIRST, d_week_seq1#41 ASC NULLS FIRST], [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] +(47) TakeOrderedAndProject +Input [10]: [s_store_name1#33, s_store_id1#35, d_week_seq1#34, (sun_sales1 / sun_sales2)#63, (mon_sales1 / mon_sales2)#64, (tue_sales1 / tue_sales2)#65, (wed_sales1 / wed_sales2)#66, (thu_sales1 / thu_sales2)#67, (fri_sales1 / fri_sales2)#68, (sat_sales1 / sat_sales2)#69] +Arguments: 100, [s_store_name1#33 ASC NULLS FIRST, s_store_id1#35 ASC NULLS FIRST, d_week_seq1#34 ASC NULLS FIRST], [s_store_name1#33, s_store_id1#35, d_week_seq1#34, (sun_sales1 / sun_sales2)#63, (mon_sales1 / mon_sales2)#64, (tue_sales1 / tue_sales2)#65, (wed_sales1 / wed_sales2)#66, (thu_sales1 / thu_sales2)#67, (fri_sales1 / fri_sales2)#68, (sat_sales1 / sat_sales2)#69] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index 9ad61e946e..6a17e8b98b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -6,24 +6,26 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] BroadcastHashJoin [ss_store_sk,s_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow InputAdapter - Exchange [d_week_seq,ss_store_sk] #1 - WholeStageCodegen (2) - HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] + CometColumnarExchange [d_week_seq,ss_store_sk] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) @@ -46,9 +48,10 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow InputAdapter - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 + CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] + ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 InputAdapter BroadcastExchange #6 WholeStageCodegen (7) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index 5f34aa3df9..6ae031c893 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -1,43 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Filter (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer_address (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometFilter (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.item (22) +* ColumnarToRow (41) ++- CometTakeOrderedAndProject (40) + +- CometFilter (39) + +- CometHashAggregate (38) + +- CometColumnarExchange (37) + +- RowToColumnar (36) + +- * HashAggregate (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer_address (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store_sales (10) + : +- ReusedExchange (16) + +- BroadcastExchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * ColumnarToRow (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) + +- CometHashAggregate (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.item (22) (unknown) Scan parquet spark_catalog.default.customer_address @@ -111,7 +113,7 @@ Join condition: None Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -(16) ReusedExchange [Reuses operator id: 44] +(16) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#9] (17) BroadcastHashJoin [codegen id : 6] @@ -154,9 +156,9 @@ Input [2]: [i_current_price#13, i_category#14] Keys [1]: [i_category#14] Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] -(25) CometExchange +(25) CometColumnarExchange Input [3]: [i_category#14, sum#15, count#16] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (26) CometHashAggregate Input [3]: [i_category#14, sum#15, count#16] @@ -205,97 +207,101 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] Results [2]: [ca_state#2, count#19] -(36) Exchange +(36) RowToColumnar Input [2]: [ca_state#2, count#19] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(37) HashAggregate [codegen id : 7] +(37) CometColumnarExchange +Input [2]: [ca_state#2, count#19] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(38) CometHashAggregate Input [2]: [ca_state#2, count#19] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#20] -Results [2]: [ca_state#2 AS state#21, count(1)#20 AS cnt#22] -(38) Filter [codegen id : 7] -Input [2]: [state#21, cnt#22] -Condition : (cnt#22 >= 10) +(39) CometFilter +Input [2]: [state#20, cnt#21] +Condition : (cnt#21 >= 10) + +(40) CometTakeOrderedAndProject +Input [2]: [state#20, cnt#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#21 ASC NULLS FIRST], output=[state#20,cnt#21]), 100, [cnt#21 ASC NULLS FIRST], [state#20, cnt#21] -(39) TakeOrderedAndProject -Input [2]: [state#21, cnt#22] -Arguments: 100, [cnt#22 ASC NULLS FIRST], [state#21, cnt#22] +(41) ColumnarToRow [codegen id : 7] +Input [2]: [state#20, cnt#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#23] +Output [2]: [d_date_sk#9, d_month_seq#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#23] -Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-subquery#24, [id=#25])) AND isnotnull(d_date_sk#9)) +(43) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#22] +Condition : ((isnotnull(d_month_seq#22) AND (d_month_seq#22 = Subquery scalar-subquery#23, [id=#24])) AND isnotnull(d_date_sk#9)) -(42) CometProject -Input [2]: [d_date_sk#9, d_month_seq#23] +(44) CometProject +Input [2]: [d_date_sk#9, d_month_seq#22] Arguments: [d_date_sk#9], [d_date_sk#9] -(43) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(44) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#24, [id=#25] -* ColumnarToRow (51) -+- CometHashAggregate (50) - +- CometExchange (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +Subquery:2 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#23, [id=#24] +* ColumnarToRow (53) ++- CometHashAggregate (52) + +- CometColumnarExchange (51) + +- CometHashAggregate (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] +Output [3]: [d_month_seq#25, d_year#26, d_moy#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(46) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) +(48) CometFilter +Input [3]: [d_month_seq#25, d_year#26, d_moy#27] +Condition : (((isnotnull(d_year#26) AND isnotnull(d_moy#27)) AND (d_year#26 = 2000)) AND (d_moy#27 = 1)) -(47) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] +(49) CometProject +Input [3]: [d_month_seq#25, d_year#26, d_moy#27] +Arguments: [d_month_seq#25], [d_month_seq#25] -(48) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +(50) CometHashAggregate +Input [1]: [d_month_seq#25] +Keys [1]: [d_month_seq#25] Functions: [] -(49) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(51) CometColumnarExchange +Input [1]: [d_month_seq#25] +Arguments: hashpartitioning(d_month_seq#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(50) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +(52) CometHashAggregate +Input [1]: [d_month_seq#25] +Keys [1]: [d_month_seq#25] Functions: [] -(51) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] +(53) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#25] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 24a40f8040..b855d42f90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -1,74 +1,76 @@ -TakeOrderedAndProject [cnt,state] - WholeStageCodegen (7) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - InputAdapter - Exchange [ca_state] #1 - WholeStageCodegen (6) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] +WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [cnt,state] + CometFilter [cnt] + CometHashAggregate [ca_state,count] + CometColumnarExchange [ca_state] #1 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ca_state,ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ca_state,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [ca_state,c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_current_addr_sk,c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometFilter [ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometColumnarExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + Project [i_item_sk] + BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow - InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [avg(i_current_price)] - CometHashAggregate [i_category,sum,count] - CometExchange [i_category] #8 - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_current_price,i_category] + CometFilter [i_current_price,i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [avg(i_current_price)] + CometHashAggregate [i_category,sum,count] + CometColumnarExchange [i_category] #8 + CometHashAggregate [i_category,i_current_price] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 8885bc8e5b..accdce467d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -1,67 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * ColumnarToRow (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * ColumnarToRow (46) - : : : +- CometFilter (45) - : : : +- CometScan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +* ColumnarToRow (67) ++- CometTakeOrderedAndProject (66) + +- CometHashAggregate (65) + +- CometColumnarExchange (64) + +- CometHashAggregate (63) + +- CometUnion (62) + :- CometHashAggregate (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- CometHashAggregate (45) + : +- CometColumnarExchange (44) + : +- RowToColumnar (43) + : +- * HashAggregate (42) + : +- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (32) + : : : : +- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) + : : : +- ReusedExchange (33) + : : +- ReusedExchange (36) + : +- ReusedExchange (39) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) + +- RowToColumnar (59) + +- * HashAggregate (58) + +- * Project (57) + +- * BroadcastHashJoin Inner BuildRight (56) + :- * Project (54) + : +- * BroadcastHashJoin Inner BuildRight (53) + : :- * Project (51) + : : +- * BroadcastHashJoin Inner BuildRight (50) + : : :- * ColumnarToRow (48) + : : : +- CometFilter (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- ReusedExchange (49) + : +- ReusedExchange (52) + +- ReusedExchange (55) (unknown) Scan parquet spark_catalog.default.store_sales @@ -79,7 +83,7 @@ Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 68] +(4) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -187,219 +191,224 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#13] Results [2]: [i_item_id#10, sum#14] -(27) Exchange +(27) RowToColumnar Input [2]: [i_item_id#10, sum#14] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) HashAggregate [codegen id : 6] +(28) CometColumnarExchange +Input [2]: [i_item_id#10, sum#14] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(29) CometHashAggregate Input [2]: [i_item_id#10, sum#14] Keys [1]: [i_item_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_item_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Output [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) +(31) CometFilter +Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] +Condition : (isnotnull(cs_bill_addr_sk#15) AND isnotnull(cs_item_sk#16)) -(31) ColumnarToRow [codegen id : 11] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +(32) ColumnarToRow [codegen id : 10] +Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -(32) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#22] +(33) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#20] -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] +(34) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(34) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] +(35) Project [codegen id : 10] +Output [3]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17] +Input [5]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#20] -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#23] +(36) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#21] -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#23] +(37) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_bill_addr_sk#15] +Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(37) Project [codegen id : 11] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] +(38) Project [codegen id : 10] +Output [2]: [cs_item_sk#16, cs_ext_sales_price#17] +Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, ca_address_sk#21] -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#24, i_item_id#25] +(39) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#22, i_item_id#23] -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#24] +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_item_sk#16] +Right keys [1]: [i_item_sk#22] Join type: Inner Join condition: None -(40) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#19, i_item_id#25] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] +(41) Project [codegen id : 10] +Output [2]: [cs_ext_sales_price#17, i_item_id#23] +Input [4]: [cs_item_sk#16, cs_ext_sales_price#17, i_item_sk#22, i_item_id#23] + +(42) HashAggregate [codegen id : 10] +Input [2]: [cs_ext_sales_price#17, i_item_id#23] +Keys [1]: [i_item_id#23] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#17))] +Aggregate Attributes [1]: [sum#24] +Results [2]: [i_item_id#23, sum#25] -(41) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#19, i_item_id#25] -Keys [1]: [i_item_id#25] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_item_id#25, sum#27] +(43) RowToColumnar +Input [2]: [i_item_id#23, sum#25] -(42) Exchange -Input [2]: [i_item_id#25, sum#27] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(44) CometColumnarExchange +Input [2]: [i_item_id#23, sum#25] +Arguments: hashpartitioning(i_item_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#25, sum#27] -Keys [1]: [i_item_id#25] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] -Results [2]: [i_item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] +(45) CometHashAggregate +Input [2]: [i_item_id#23, sum#25] +Keys [1]: [i_item_id#23] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#17))] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) +(47) CometFilter +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) -(46) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +(48) ColumnarToRow [codegen id : 15] +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -(47) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#35] +(49) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#31] -(48) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#35] +(50) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#29] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(49) Project [codegen id : 17] -Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] +(51) Project [codegen id : 15] +Output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] +Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] -(50) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#36] +(52) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#32] -(51) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#31] -Right keys [1]: [ca_address_sk#36] +(53) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_bill_addr_sk#27] +Right keys [1]: [ca_address_sk#32] Join type: Inner Join condition: None -(52) Project [codegen id : 17] -Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] +(54) Project [codegen id : 15] +Output [2]: [ws_item_sk#26, ws_ext_sales_price#28] +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] -(53) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#37, i_item_id#38] +(55) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#33, i_item_id#34] -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#30] -Right keys [1]: [i_item_sk#37] +(56) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#33] Join type: Inner Join condition: None -(55) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#32, i_item_id#38] -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] +(57) Project [codegen id : 15] +Output [2]: [ws_ext_sales_price#28, i_item_id#34] +Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_item_id#34] -(56) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#32, i_item_id#38] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum#39] -Results [2]: [i_item_id#38, sum#40] +(58) HashAggregate [codegen id : 15] +Input [2]: [ws_ext_sales_price#28, i_item_id#34] +Keys [1]: [i_item_id#34] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] +Aggregate Attributes [1]: [sum#35] +Results [2]: [i_item_id#34, sum#36] -(57) Exchange -Input [2]: [i_item_id#38, sum#40] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(59) RowToColumnar +Input [2]: [i_item_id#34, sum#36] -(58) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#38, sum#40] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] -Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] +(60) CometColumnarExchange +Input [2]: [i_item_id#34, sum#36] +Arguments: hashpartitioning(i_item_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(59) Union +(61) CometHashAggregate +Input [2]: [i_item_id#34, sum#36] +Keys [1]: [i_item_id#34] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -(60) HashAggregate [codegen id : 19] -Input [2]: [i_item_id#10, total_sales#16] +(62) CometUnion +Child 0 Input [2]: [i_item_id#10, total_sales#37] +Child 1 Input [2]: [i_item_id#23, total_sales#38] +Child 2 Input [2]: [i_item_id#34, total_sales#39] + +(63) CometHashAggregate +Input [2]: [i_item_id#10, total_sales#37] Keys [1]: [i_item_id#10] -Functions [1]: [partial_sum(total_sales#16)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [3]: [i_item_id#10, sum#45, isEmpty#46] +Functions [1]: [partial_sum(total_sales#37)] -(61) Exchange -Input [3]: [i_item_id#10, sum#45, isEmpty#46] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(64) CometColumnarExchange +Input [3]: [i_item_id#10, sum#40, isEmpty#41] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(62) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#10, sum#45, isEmpty#46] +(65) CometHashAggregate +Input [3]: [i_item_id#10, sum#40, isEmpty#41] Keys [1]: [i_item_id#10] -Functions [1]: [sum(total_sales#16)] -Aggregate Attributes [1]: [sum(total_sales#16)#47] -Results [2]: [i_item_id#10, sum(total_sales#16)#47 AS total_sales#48] +Functions [1]: [sum(total_sales#37)] + +(66) CometTakeOrderedAndProject +Input [2]: [i_item_id#10, total_sales#42] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#10 ASC NULLS FIRST,total_sales#42 ASC NULLS FIRST], output=[i_item_id#10,total_sales#42]), 100, [i_item_id#10 ASC NULLS FIRST, total_sales#42 ASC NULLS FIRST], [i_item_id#10, total_sales#42] -(63) TakeOrderedAndProject -Input [2]: [i_item_id#10, total_sales#48] -Arguments: 100, [i_item_id#10 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#10, total_sales#48] +(67) ColumnarToRow [codegen id : 16] +Input [2]: [i_item_id#10, total_sales#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (68) -+- * ColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#49, d_moy#50] +Output [3]: [d_date_sk#6, d_year#43, d_moy#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] ReadSchema: struct -(65) CometFilter -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 9)) AND isnotnull(d_date_sk#6)) +(69) CometFilter +Input [3]: [d_date_sk#6, d_year#43, d_moy#44] +Condition : ((((isnotnull(d_year#43) AND isnotnull(d_moy#44)) AND (d_year#43 = 1998)) AND (d_moy#44 = 9)) AND isnotnull(d_date_sk#6)) -(66) CometProject -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +(70) CometProject +Input [3]: [d_date_sk#6, d_year#43, d_moy#44] Arguments: [d_date_sk#6], [d_date_sk#6] -(67) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index b010414a86..bad7cbf2b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -1,105 +1,101 @@ -TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (20) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #2 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,total_sales] + CometHashAggregate [i_item_id,sum,isEmpty] + CometColumnarExchange [i_item_id] #1 + CometHashAggregate [i_item_id,total_sales] + CometUnion + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #6 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_item_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [i_item_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_item_id,i_category] + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #7 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [ca_address_sk] #4 + ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #8 + RowToColumnar + WholeStageCodegen (15) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [ca_address_sk] #4 + ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index ee9a8409a2..69767b1f68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -1,71 +1,75 @@ == Physical Plan == -* Project (67) -+- * BroadcastNestedLoopJoin Inner BuildRight (66) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (26) - : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : :- * Project (20) - : : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * ColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan parquet spark_catalog.default.promotion (11) - : : : : +- ReusedExchange (18) - : : : +- BroadcastExchange (24) - : : : +- * ColumnarToRow (23) - : : : +- CometFilter (22) - : : : +- CometScan parquet spark_catalog.default.customer (21) - : : +- BroadcastExchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.customer_address (27) - : +- BroadcastExchange (38) - : +- * ColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan parquet spark_catalog.default.item (34) - +- BroadcastExchange (65) - +- * HashAggregate (64) - +- Exchange (63) - +- * HashAggregate (62) - +- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * Project (55) - : : +- * BroadcastHashJoin Inner BuildRight (54) - : : :- * Project (52) - : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : :- * Project (49) - : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : :- * ColumnarToRow (46) - : : : : : +- CometFilter (45) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (44) - : : : : +- ReusedExchange (47) - : : : +- ReusedExchange (50) - : : +- ReusedExchange (53) - : +- ReusedExchange (56) - +- ReusedExchange (59) +* Project (71) ++- * BroadcastNestedLoopJoin Inner BuildRight (70) + :- * ColumnarToRow (45) + : +- CometHashAggregate (44) + : +- CometColumnarExchange (43) + : +- RowToColumnar (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (33) + : : +- * BroadcastHashJoin Inner BuildRight (32) + : : :- * Project (26) + : : : +- * BroadcastHashJoin Inner BuildRight (25) + : : : :- * Project (20) + : : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * ColumnarToRow (7) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- CometProject (13) + : : : : : +- CometFilter (12) + : : : : : +- CometScan parquet spark_catalog.default.promotion (11) + : : : : +- ReusedExchange (18) + : : : +- BroadcastExchange (24) + : : : +- * ColumnarToRow (23) + : : : +- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.customer (21) + : : +- BroadcastExchange (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.customer_address (27) + : +- BroadcastExchange (38) + : +- * ColumnarToRow (37) + : +- CometProject (36) + : +- CometFilter (35) + : +- CometScan parquet spark_catalog.default.item (34) + +- BroadcastExchange (69) + +- * ColumnarToRow (68) + +- CometHashAggregate (67) + +- CometColumnarExchange (66) + +- RowToColumnar (65) + +- * HashAggregate (64) + +- * Project (63) + +- * BroadcastHashJoin Inner BuildRight (62) + :- * Project (60) + : +- * BroadcastHashJoin Inner BuildRight (59) + : :- * Project (57) + : : +- * BroadcastHashJoin Inner BuildRight (56) + : : :- * Project (54) + : : : +- * BroadcastHashJoin Inner BuildRight (53) + : : : :- * Project (51) + : : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : : :- * ColumnarToRow (48) + : : : : : +- CometFilter (47) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (46) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (58) + +- ReusedExchange (61) (unknown) Scan parquet spark_catalog.default.store_sales @@ -147,7 +151,7 @@ Join condition: None Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] -(18) ReusedExchange [Reuses operator id: 72] +(18) ReusedExchange [Reuses operator id: 76] Output [1]: [d_date_sk#14] (19) BroadcastHashJoin [codegen id : 7] @@ -259,159 +263,167 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#21] Results [1]: [sum#22] -(42) Exchange +(42) RowToColumnar Input [1]: [sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(43) HashAggregate [codegen id : 15] +(43) CometColumnarExchange +Input [1]: [sum#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(44) CometHashAggregate Input [1]: [sum#22] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#23,17,2) AS promotions#24] + +(45) ColumnarToRow [codegen id : 15] +Input [1]: [promotions#23] (unknown) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Output [5]: [ss_item_sk#24, ss_customer_sk#25, ss_store_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(45) CometFilter -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) +(47) CometFilter +Input [5]: [ss_item_sk#24, ss_customer_sk#25, ss_store_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Condition : ((isnotnull(ss_store_sk#26) AND isnotnull(ss_customer_sk#25)) AND isnotnull(ss_item_sk#24)) -(46) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +(48) ColumnarToRow [codegen id : 13] +Input [5]: [ss_item_sk#24, ss_customer_sk#25, ss_store_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -(47) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#31] +(49) ReusedExchange [Reuses operator id: 8] +Output [1]: [s_store_sk#30] -(48) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#27] -Right keys [1]: [s_store_sk#31] +(50) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_store_sk#26] +Right keys [1]: [s_store_sk#30] Join type: Inner Join condition: None -(49) Project [codegen id : 13] -Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] +(51) Project [codegen id : 13] +Output [4]: [ss_item_sk#24, ss_customer_sk#25, ss_ext_sales_price#27, ss_sold_date_sk#28] +Input [6]: [ss_item_sk#24, ss_customer_sk#25, ss_store_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, s_store_sk#30] -(50) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#32] +(52) ReusedExchange [Reuses operator id: 76] +Output [1]: [d_date_sk#31] -(51) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#32] +(53) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_sold_date_sk#28] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(52) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] +(54) Project [codegen id : 13] +Output [3]: [ss_item_sk#24, ss_customer_sk#25, ss_ext_sales_price#27] +Input [5]: [ss_item_sk#24, ss_customer_sk#25, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#31] -(53) ReusedExchange [Reuses operator id: 24] -Output [2]: [c_customer_sk#33, c_current_addr_sk#34] +(55) ReusedExchange [Reuses operator id: 24] +Output [2]: [c_customer_sk#32, c_current_addr_sk#33] -(54) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#33] +(56) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_customer_sk#25] +Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(55) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] +(57) Project [codegen id : 13] +Output [3]: [ss_item_sk#24, ss_ext_sales_price#27, c_current_addr_sk#33] +Input [5]: [ss_item_sk#24, ss_customer_sk#25, ss_ext_sales_price#27, c_customer_sk#32, c_current_addr_sk#33] -(56) ReusedExchange [Reuses operator id: 31] -Output [1]: [ca_address_sk#35] +(58) ReusedExchange [Reuses operator id: 31] +Output [1]: [ca_address_sk#34] -(57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#35] +(59) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [c_current_addr_sk#33] +Right keys [1]: [ca_address_sk#34] Join type: Inner Join condition: None -(58) Project [codegen id : 13] -Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] +(60) Project [codegen id : 13] +Output [2]: [ss_item_sk#24, ss_ext_sales_price#27] +Input [4]: [ss_item_sk#24, ss_ext_sales_price#27, c_current_addr_sk#33, ca_address_sk#34] -(59) ReusedExchange [Reuses operator id: 38] -Output [1]: [i_item_sk#36] +(61) ReusedExchange [Reuses operator id: 38] +Output [1]: [i_item_sk#35] -(60) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#25] -Right keys [1]: [i_item_sk#36] +(62) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_item_sk#24] +Right keys [1]: [i_item_sk#35] Join type: Inner Join condition: None -(61) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#28] -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] +(63) Project [codegen id : 13] +Output [1]: [ss_ext_sales_price#27] +Input [3]: [ss_item_sk#24, ss_ext_sales_price#27, i_item_sk#35] -(62) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#28] +(64) HashAggregate [codegen id : 13] +Input [1]: [ss_ext_sales_price#27] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum#37] -Results [1]: [sum#38] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] +Aggregate Attributes [1]: [sum#36] +Results [1]: [sum#37] -(63) Exchange -Input [1]: [sum#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(65) RowToColumnar +Input [1]: [sum#37] -(64) HashAggregate [codegen id : 14] -Input [1]: [sum#38] +(66) CometColumnarExchange +Input [1]: [sum#37] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(67) CometHashAggregate +Input [1]: [sum#37] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#28))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#39,17,2) AS total#40] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] + +(68) ColumnarToRow [codegen id : 14] +Input [1]: [total#38] -(65) BroadcastExchange -Input [1]: [total#40] +(69) BroadcastExchange +Input [1]: [total#38] Arguments: IdentityBroadcastMode, [plan_id=8] -(66) BroadcastNestedLoopJoin [codegen id : 15] +(70) BroadcastNestedLoopJoin [codegen id : 15] Join type: Inner Join condition: None -(67) Project [codegen id : 15] -Output [3]: [promotions#24, total#40, ((cast(promotions#24 as decimal(15,4)) / cast(total#40 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#41] -Input [2]: [promotions#24, total#40] +(71) Project [codegen id : 15] +Output [3]: [promotions#23, total#38, ((cast(promotions#23 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] +Input [2]: [promotions#23, total#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#42, d_moy#43] +Output [3]: [d_date_sk#14, d_year#40, d_moy#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter -Input [3]: [d_date_sk#14, d_year#42, d_moy#43] -Condition : ((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 1998)) AND (d_moy#43 = 11)) AND isnotnull(d_date_sk#14)) +(73) CometFilter +Input [3]: [d_date_sk#14, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 1998)) AND (d_moy#41 = 11)) AND isnotnull(d_date_sk#14)) -(70) CometProject -Input [3]: [d_date_sk#14, d_year#42, d_moy#43] +(74) CometProject +Input [3]: [d_date_sk#14, d_year#40, d_moy#41] Arguments: [d_date_sk#14], [d_date_sk#14] -(71) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(72) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 44 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 46 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index 2c3d07ac64..ddedbe1379 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -1,106 +1,110 @@ WholeStageCodegen (15) Project [promotions,total] BroadcastNestedLoopJoin - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (7) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometHashAggregate [sum] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_gmt_offset,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] + CometProject [p_promo_sk] + CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #6 + WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometProject [i_item_sk] + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #8 WholeStageCodegen (14) - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] + ColumnarToRow InputAdapter - Exchange #9 - WholeStageCodegen (13) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometHashAggregate [sum] + CometColumnarExchange #9 + RowToColumnar + WholeStageCodegen (13) + HashAggregate [ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [s_store_sk] #3 InputAdapter - ReusedExchange [s_store_sk] #3 + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [c_customer_sk,c_current_addr_sk] #5 InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk] #5 + ReusedExchange [ca_address_sk] #6 InputAdapter - ReusedExchange [ca_address_sk] #6 - InputAdapter - ReusedExchange [i_item_sk] #7 + ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt index 82ded1c507..427bce7dff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt @@ -1,36 +1,38 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.ship_mode (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.web_site (16) - +- BroadcastExchange (26) - +- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +* ColumnarToRow (34) ++- CometTakeOrderedAndProject (33) + +- CometHashAggregate (32) + +- CometColumnarExchange (31) + +- RowToColumnar (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.warehouse (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.ship_mode (10) + : +- BroadcastExchange (19) + : +- * ColumnarToRow (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.web_site (16) + +- BroadcastExchange (26) + +- * ColumnarToRow (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) (unknown) Scan parquet spark_catalog.default.web_sales @@ -170,18 +172,22 @@ Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) < Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] Results [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -(30) Exchange +(30) RowToColumnar Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, web_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(31) HashAggregate [codegen id : 6] +(31) CometColumnarExchange +Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, web_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(32) CometHashAggregate Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [3]: [_groupingexpression#14, sm_type#9, web_name#11] Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] -Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] -(32) TakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] -Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] +(33) CometTakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#25, sm_type#9, web_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#25 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,web_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#25,sm_type#9,web_name#11,30 days #26,31 - 60 days #27,61 - 90 days #28,91 - 120 days #29,>120 days #30]), 100, [substr(w_warehouse_name, 1, 20)#25 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#25, sm_type#9, web_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] + +(34) ColumnarToRow [codegen id : 6] +Input [8]: [substr(w_warehouse_name, 1, 20)#25, sm_type#9, web_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt index 5ae522ce1c..4fe012742f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt @@ -1,48 +1,50 @@ -TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) - HashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - InputAdapter - Exchange [_groupingexpression,sm_type,web_name] #1 - WholeStageCodegen (5) - HashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,w_warehouse_name] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - Project [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] +WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] + CometColumnarExchange [_groupingexpression,sm_type,web_name] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,w_warehouse_name] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Project [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometFilter [sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index 458a35b388..18ed395704 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -1,32 +1,34 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * Project (27) - +- * Filter (26) - +- Window (25) - +- * Sort (24) - +- Exchange (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.store (14) +TakeOrderedAndProject (30) ++- * Project (29) + +- * Filter (28) + +- Window (27) + +- * ColumnarToRow (26) + +- CometSort (25) + +- CometColumnarExchange (24) + +- CometHashAggregate (23) + +- CometColumnarExchange (22) + +- RowToColumnar (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.store_sales (5) + : +- ReusedExchange (11) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) (unknown) Scan parquet spark_catalog.default.item @@ -76,7 +78,7 @@ Join condition: None Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -(11) ReusedExchange [Reuses operator id: 33] +(11) ReusedExchange [Reuses operator id: 35] Output [2]: [d_date_sk#15, d_moy#16] (12) BroadcastHashJoin [codegen id : 4] @@ -124,70 +126,74 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] Aggregate Attributes [1]: [sum#18] Results [3]: [i_manager_id#5, d_moy#16, sum#19] -(21) Exchange +(21) RowToColumnar Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) HashAggregate [codegen id : 5] +(22) CometColumnarExchange +Input [3]: [i_manager_id#5, d_moy#16, sum#19] +Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(23) CometHashAggregate Input [3]: [i_manager_id#5, d_moy#16, sum#19] Keys [2]: [i_manager_id#5, d_moy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] -(23) Exchange -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(24) CometColumnarExchange +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(25) CometSort +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] -(24) Sort [codegen id : 6] -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [i_manager_id#5 ASC NULLS FIRST], false, 0 +(26) ColumnarToRow [codegen id : 5] +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -(25) Window -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] +(27) Window +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] -(26) Filter [codegen id : 7] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END +(28) Filter [codegen id : 6] +Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] +Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END -(27) Project [codegen id : 7] -Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] +(29) Project [codegen id : 6] +Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -(28) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +(30) TakeOrderedAndProject +Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] +Output [3]: [d_date_sk#15, d_month_seq#23, d_moy#16] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(30) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) +(32) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#23, d_moy#16] +Condition : (d_month_seq#23 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(31) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] +(33) CometProject +Input [3]: [d_date_sk#15, d_month_seq#23, d_moy#16] Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] -(32) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#16] -(33) BroadcastExchange +(35) BroadcastExchange Input [2]: [d_date_sk#15, d_moy#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index 7f6f8c1370..cd9cffbd1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -1,17 +1,17 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (7) + WholeStageCodegen (6) Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] - WholeStageCodegen (6) - Sort [i_manager_id] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [i_manager_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_manager_id,d_moy] #2 + CometSort [i_manager_id] + CometColumnarExchange [i_manager_id] #1 + CometHashAggregate [i_manager_id,d_moy,sum] + CometColumnarExchange [i_manager_id,d_moy] #2 + RowToColumnar WholeStageCodegen (4) HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] Project [i_manager_id,ss_sales_price,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index c21eda6be3..c4cbdaa3b5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -1,183 +1,189 @@ == Physical Plan == -* Sort (179) -+- Exchange (178) - +- * Project (177) - +- * SortMergeJoin Inner (176) - :- * Sort (108) - : +- Exchange (107) - : +- * HashAggregate (106) - : +- * HashAggregate (105) - : +- * Project (104) - : +- * BroadcastHashJoin Inner BuildRight (103) - : :- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * Project (94) - : : : +- * BroadcastHashJoin Inner BuildRight (93) - : : : :- * Project (88) - : : : : +- * BroadcastHashJoin Inner BuildRight (87) - : : : : :- * Project (85) - : : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : : :- * Project (79) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : :- * Project (76) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : : : : : :- * Project (70) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (69) - : : : : : : : : :- * Project (64) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : : : : : : :- * Project (61) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : : : : : : : :- * Project (55) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : : : : : : : :- * Project (52) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : : : : : : : : : : :- * Project (46) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : : : : : : : : : : :- * Project (40) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : : : : : : : : : : : : :- * Project (34) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : : : : : : : : : : : :- * ColumnarToRow (31) - : : : : : : : : : : : : : : : : +- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- ReusedExchange (32) - : : : : : : : : : : : : : : +- BroadcastExchange (38) - : : : : : : : : : : : : : : +- * ColumnarToRow (37) - : : : : : : : : : : : : : : +- CometFilter (36) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (35) - : : : : : : : : : : : : : +- BroadcastExchange (44) - : : : : : : : : : : : : : +- * ColumnarToRow (43) - : : : : : : : : : : : : : +- CometFilter (42) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) - : : : : : : : : : : : : +- BroadcastExchange (50) - : : : : : : : : : : : : +- * ColumnarToRow (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (53) - : : : : : : : : : : +- BroadcastExchange (59) - : : : : : : : : : : +- * ColumnarToRow (58) - : : : : : : : : : : +- CometFilter (57) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (56) - : : : : : : : : : +- ReusedExchange (62) - : : : : : : : : +- BroadcastExchange (68) - : : : : : : : : +- * ColumnarToRow (67) - : : : : : : : : +- CometFilter (66) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (65) - : : : : : : : +- BroadcastExchange (74) - : : : : : : : +- * ColumnarToRow (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (77) - : : : : : +- BroadcastExchange (83) - : : : : : +- * ColumnarToRow (82) - : : : : : +- CometFilter (81) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (80) - : : : : +- ReusedExchange (86) - : : : +- BroadcastExchange (92) - : : : +- * ColumnarToRow (91) - : : : +- CometFilter (90) - : : : +- CometScan parquet spark_catalog.default.income_band (89) - : : +- ReusedExchange (95) - : +- BroadcastExchange (102) - : +- * ColumnarToRow (101) - : +- CometProject (100) - : +- CometFilter (99) - : +- CometScan parquet spark_catalog.default.item (98) - +- * Sort (175) - +- Exchange (174) - +- * HashAggregate (173) - +- * HashAggregate (172) - +- * Project (171) - +- * BroadcastHashJoin Inner BuildRight (170) - :- * Project (168) - : +- * BroadcastHashJoin Inner BuildRight (167) - : :- * Project (165) - : : +- * BroadcastHashJoin Inner BuildRight (164) - : : :- * Project (162) - : : : +- * BroadcastHashJoin Inner BuildRight (161) - : : : :- * Project (159) - : : : : +- * BroadcastHashJoin Inner BuildRight (158) - : : : : :- * Project (156) - : : : : : +- * BroadcastHashJoin Inner BuildRight (155) - : : : : : :- * Project (153) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (152) - : : : : : : :- * Project (150) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (149) - : : : : : : : :- * Project (147) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (146) - : : : : : : : : :- * Project (144) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (143) - : : : : : : : : : :- * Project (141) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (140) - : : : : : : : : : : :- * Project (138) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (137) - : : : : : : : : : : : :- * Project (135) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (134) - : : : : : : : : : : : : :- * Project (132) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (131) - : : : : : : : : : : : : : :- * Project (129) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (128) - : : : : : : : : : : : : : : :- * ColumnarToRow (126) - : : : : : : : : : : : : : : : +- CometProject (125) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (124) - : : : : : : : : : : : : : : : :- CometSort (118) - : : : : : : : : : : : : : : : : +- CometExchange (117) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (115) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (111) - : : : : : : : : : : : : : : : : : +- CometFilter (110) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (109) - : : : : : : : : : : : : : : : : +- CometProject (114) - : : : : : : : : : : : : : : : : +- CometFilter (113) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (112) - : : : : : : : : : : : : : : : +- CometSort (123) - : : : : : : : : : : : : : : : +- CometProject (122) - : : : : : : : : : : : : : : : +- CometFilter (121) - : : : : : : : : : : : : : : : +- CometHashAggregate (120) - : : : : : : : : : : : : : : : +- ReusedExchange (119) - : : : : : : : : : : : : : : +- ReusedExchange (127) - : : : : : : : : : : : : : +- ReusedExchange (130) - : : : : : : : : : : : : +- ReusedExchange (133) - : : : : : : : : : : : +- ReusedExchange (136) - : : : : : : : : : : +- ReusedExchange (139) - : : : : : : : : : +- ReusedExchange (142) - : : : : : : : : +- ReusedExchange (145) - : : : : : : : +- ReusedExchange (148) - : : : : : : +- ReusedExchange (151) - : : : : : +- ReusedExchange (154) - : : : : +- ReusedExchange (157) - : : : +- ReusedExchange (160) - : : +- ReusedExchange (163) - : +- ReusedExchange (166) - +- ReusedExchange (169) +* ColumnarToRow (185) ++- CometSort (184) + +- CometColumnarExchange (183) + +- RowToColumnar (182) + +- * Project (181) + +- * SortMergeJoin Inner (180) + :- * ColumnarToRow (110) + : +- CometSort (109) + : +- CometColumnarExchange (108) + : +- RowToColumnar (107) + : +- * HashAggregate (106) + : +- * HashAggregate (105) + : +- * Project (104) + : +- * BroadcastHashJoin Inner BuildRight (103) + : :- * Project (97) + : : +- * BroadcastHashJoin Inner BuildRight (96) + : : :- * Project (94) + : : : +- * BroadcastHashJoin Inner BuildRight (93) + : : : :- * Project (88) + : : : : +- * BroadcastHashJoin Inner BuildRight (87) + : : : : :- * Project (85) + : : : : : +- * BroadcastHashJoin Inner BuildRight (84) + : : : : : :- * Project (79) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) + : : : : : : :- * Project (76) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) + : : : : : : : :- * Project (70) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (69) + : : : : : : : : :- * Project (64) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : : : : : : : :- * Project (61) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : : : : : : : :- * Project (55) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : : : : : : : :- * Project (52) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : : : : : : : : : :- * Project (46) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : : : : : : : : : : : :- * Project (40) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (39) + : : : : : : : : : : : : : : :- * Project (34) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : : : : : : : : : : : : :- * ColumnarToRow (31) + : : : : : : : : : : : : : : : : +- CometProject (30) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) + : : : : : : : : : : : : : : : : :- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- CometSort (28) + : : : : : : : : : : : : : : : : +- CometProject (27) + : : : : : : : : : : : : : : : : +- CometFilter (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (24) + : : : : : : : : : : : : : : : : +- CometHashAggregate (23) + : : : : : : : : : : : : : : : : +- CometProject (22) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) + : : : : : : : : : : : : : : : : :- CometSort (15) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (14) + : : : : : : : : : : : : : : : : : +- CometProject (13) + : : : : : : : : : : : : : : : : : +- CometFilter (12) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : : : : : : : : : : : +- CometSort (20) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (19) + : : : : : : : : : : : : : : : : +- CometProject (18) + : : : : : : : : : : : : : : : : +- CometFilter (17) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : : : : : : : : : : : : : : : +- ReusedExchange (32) + : : : : : : : : : : : : : : +- BroadcastExchange (38) + : : : : : : : : : : : : : : +- * ColumnarToRow (37) + : : : : : : : : : : : : : : +- CometFilter (36) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (35) + : : : : : : : : : : : : : +- BroadcastExchange (44) + : : : : : : : : : : : : : +- * ColumnarToRow (43) + : : : : : : : : : : : : : +- CometFilter (42) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) + : : : : : : : : : : : : +- BroadcastExchange (50) + : : : : : : : : : : : : +- * ColumnarToRow (49) + : : : : : : : : : : : : +- CometFilter (48) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (47) + : : : : : : : : : : : +- ReusedExchange (53) + : : : : : : : : : : +- BroadcastExchange (59) + : : : : : : : : : : +- * ColumnarToRow (58) + : : : : : : : : : : +- CometFilter (57) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (56) + : : : : : : : : : +- ReusedExchange (62) + : : : : : : : : +- BroadcastExchange (68) + : : : : : : : : +- * ColumnarToRow (67) + : : : : : : : : +- CometFilter (66) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (65) + : : : : : : : +- BroadcastExchange (74) + : : : : : : : +- * ColumnarToRow (73) + : : : : : : : +- CometFilter (72) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (71) + : : : : : : +- ReusedExchange (77) + : : : : : +- BroadcastExchange (83) + : : : : : +- * ColumnarToRow (82) + : : : : : +- CometFilter (81) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (80) + : : : : +- ReusedExchange (86) + : : : +- BroadcastExchange (92) + : : : +- * ColumnarToRow (91) + : : : +- CometFilter (90) + : : : +- CometScan parquet spark_catalog.default.income_band (89) + : : +- ReusedExchange (95) + : +- BroadcastExchange (102) + : +- * ColumnarToRow (101) + : +- CometProject (100) + : +- CometFilter (99) + : +- CometScan parquet spark_catalog.default.item (98) + +- * ColumnarToRow (179) + +- CometSort (178) + +- CometColumnarExchange (177) + +- RowToColumnar (176) + +- * HashAggregate (175) + +- * HashAggregate (174) + +- * Project (173) + +- * BroadcastHashJoin Inner BuildRight (172) + :- * Project (170) + : +- * BroadcastHashJoin Inner BuildRight (169) + : :- * Project (167) + : : +- * BroadcastHashJoin Inner BuildRight (166) + : : :- * Project (164) + : : : +- * BroadcastHashJoin Inner BuildRight (163) + : : : :- * Project (161) + : : : : +- * BroadcastHashJoin Inner BuildRight (160) + : : : : :- * Project (158) + : : : : : +- * BroadcastHashJoin Inner BuildRight (157) + : : : : : :- * Project (155) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) + : : : : : : :- * Project (152) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) + : : : : : : : :- * Project (149) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) + : : : : : : : : :- * Project (146) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) + : : : : : : : : : :- * Project (143) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) + : : : : : : : : : : :- * Project (140) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) + : : : : : : : : : : : :- * Project (137) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) + : : : : : : : : : : : : :- * Project (134) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) + : : : : : : : : : : : : : :- * Project (131) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) + : : : : : : : : : : : : : : :- * ColumnarToRow (128) + : : : : : : : : : : : : : : : +- CometProject (127) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (126) + : : : : : : : : : : : : : : : :- CometSort (120) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (119) + : : : : : : : : : : : : : : : : +- CometProject (118) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) + : : : : : : : : : : : : : : : : : +- CometFilter (112) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) + : : : : : : : : : : : : : : : : +- CometProject (116) + : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) + : : : : : : : : : : : : : : : +- CometSort (125) + : : : : : : : : : : : : : : : +- CometProject (124) + : : : : : : : : : : : : : : : +- CometFilter (123) + : : : : : : : : : : : : : : : +- CometHashAggregate (122) + : : : : : : : : : : : : : : : +- ReusedExchange (121) + : : : : : : : : : : : : : : +- ReusedExchange (129) + : : : : : : : : : : : : : +- ReusedExchange (132) + : : : : : : : : : : : : +- ReusedExchange (135) + : : : : : : : : : : : +- ReusedExchange (138) + : : : : : : : : : : +- ReusedExchange (141) + : : : : : : : : : +- ReusedExchange (144) + : : : : : : : : +- ReusedExchange (147) + : : : : : : : +- ReusedExchange (150) + : : : : : : +- ReusedExchange (153) + : : : : : +- ReusedExchange (156) + : : : : +- ReusedExchange (159) + : : : +- ReusedExchange (162) + : : +- ReusedExchange (165) + : +- ReusedExchange (168) + +- ReusedExchange (171) (unknown) Scan parquet spark_catalog.default.store_sales @@ -220,9 +226,9 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) CometExchange +(9) CometColumnarExchange Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] (10) CometSort Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] @@ -243,9 +249,9 @@ Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(14) CometExchange +(14) CometColumnarExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (15) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] @@ -266,9 +272,9 @@ Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(19) CometExchange +(19) CometColumnarExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (20) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] @@ -288,9 +294,9 @@ Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reverse Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(24) CometExchange +(24) CometColumnarExchange Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] (25) CometHashAggregate Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] @@ -321,7 +327,7 @@ Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_add (31) ColumnarToRow [codegen id : 16] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(32) ReusedExchange [Reuses operator id: 183] +(32) ReusedExchange [Reuses operator id: 189] Output [2]: [d_date_sk#32, d_year#33] (33) BroadcastHashJoin [codegen id : 16] @@ -669,13 +675,19 @@ Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledV Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#9))#81, sum(UnscaledValue(ss_list_price#10))#82, sum(UnscaledValue(ss_coupon_amt#11))#83] Results [17]: [i_product_name#71 AS product_name#84, i_item_sk#68 AS item_sk#85, s_store_name#35 AS store_name#86, s_zip#36 AS store_zip#87, ca_street_number#57 AS b_street_number#88, ca_street_name#58 AS b_streen_name#89, ca_city#59 AS b_city#90, ca_zip#60 AS b_zip#91, ca_street_number#62 AS c_street_number#92, ca_street_name#63 AS c_street_name#93, ca_city#64 AS c_city#94, ca_zip#65 AS c_zip#95, d_year#33 AS syear#96, count(1)#80 AS cnt#97, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#81,17,2) AS s1#98, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#82,17,2) AS s2#99, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#83,17,2) AS s3#100] -(107) Exchange +(107) RowToColumnar Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(108) Sort [codegen id : 17] +(108) CometColumnarExchange +Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] +Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(109) CometSort +Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] +Arguments: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100], [item_sk#85 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, store_zip#87 ASC NULLS FIRST] + +(110) ColumnarToRow [codegen id : 17] Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: [item_sk#85 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, store_zip#87 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_sales Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] @@ -685,11 +697,11 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#112), dynamicpruningexpression(ss_s PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(110) CometFilter +(112) CometFilter Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Condition : (((((((isnotnull(ss_item_sk#101) AND isnotnull(ss_ticket_number#108)) AND isnotnull(ss_store_sk#106)) AND isnotnull(ss_customer_sk#102)) AND isnotnull(ss_cdemo_sk#103)) AND isnotnull(ss_promo_sk#107)) AND isnotnull(ss_hdemo_sk#104)) AND isnotnull(ss_addr_sk#105)) -(111) CometBroadcastExchange +(113) CometBroadcastExchange Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] @@ -700,305 +712,317 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(113) CometFilter +(115) CometFilter Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] Condition : (isnotnull(sr_item_sk#114) AND isnotnull(sr_ticket_number#115)) -(114) CometProject +(116) CometProject Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] Arguments: [sr_item_sk#114, sr_ticket_number#115], [sr_item_sk#114, sr_ticket_number#115] -(115) CometBroadcastHashJoin +(117) CometBroadcastHashJoin Left output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Right output [2]: [sr_item_sk#114, sr_ticket_number#115] Arguments: [ss_item_sk#101, ss_ticket_number#108], [sr_item_sk#114, sr_ticket_number#115], Inner -(116) CometProject +(118) CometProject Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, sr_item_sk#114, sr_ticket_number#115] Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -(117) CometExchange +(119) CometColumnarExchange Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] +Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(118) CometSort +(120) CometSort Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101 ASC NULLS FIRST] -(119) ReusedExchange [Reuses operator id: 24] +(121) ReusedExchange [Reuses operator id: 24] Output [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] -(120) CometHashAggregate +(122) CometHashAggregate Input [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] Keys [1]: [cs_item_sk#117] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#121)), sum(((cr_refunded_cash#122 + cr_reversed_charge#123) + cr_store_credit#124))] -(121) CometFilter +(123) CometFilter Input [3]: [cs_item_sk#117, sale#30, refund#31] Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(122) CometProject +(124) CometProject Input [3]: [cs_item_sk#117, sale#30, refund#31] Arguments: [cs_item_sk#117], [cs_item_sk#117] -(123) CometSort +(125) CometSort Input [1]: [cs_item_sk#117] Arguments: [cs_item_sk#117], [cs_item_sk#117 ASC NULLS FIRST] -(124) CometSortMergeJoin +(126) CometSortMergeJoin Left output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Right output [1]: [cs_item_sk#117] Arguments: [ss_item_sk#101], [cs_item_sk#117], Inner -(125) CometProject +(127) CometProject Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, cs_item_sk#117] Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -(126) ColumnarToRow [codegen id : 33] +(128) ColumnarToRow [codegen id : 33] Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -(127) ReusedExchange [Reuses operator id: 187] +(129) ReusedExchange [Reuses operator id: 193] Output [2]: [d_date_sk#125, d_year#126] -(128) BroadcastHashJoin [codegen id : 33] +(130) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_sold_date_sk#112] Right keys [1]: [d_date_sk#125] Join type: Inner Join condition: None -(129) Project [codegen id : 33] +(131) Project [codegen id : 33] Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126] Input [13]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, d_date_sk#125, d_year#126] -(130) ReusedExchange [Reuses operator id: 38] +(132) ReusedExchange [Reuses operator id: 38] Output [3]: [s_store_sk#127, s_store_name#128, s_zip#129] -(131) BroadcastHashJoin [codegen id : 33] +(133) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_store_sk#106] Right keys [1]: [s_store_sk#127] Join type: Inner Join condition: None -(132) Project [codegen id : 33] +(134) Project [codegen id : 33] Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129] Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_sk#127, s_store_name#128, s_zip#129] -(133) ReusedExchange [Reuses operator id: 44] +(135) ReusedExchange [Reuses operator id: 44] Output [6]: [c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -(134) BroadcastHashJoin [codegen id : 33] +(136) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_customer_sk#102] Right keys [1]: [c_customer_sk#130] Join type: Inner Join condition: None -(135) Project [codegen id : 33] +(137) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] Input [18]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -(136) ReusedExchange [Reuses operator id: 50] +(138) ReusedExchange [Reuses operator id: 50] Output [2]: [d_date_sk#136, d_year#137] -(137) BroadcastHashJoin [codegen id : 33] +(139) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_first_sales_date_sk#135] Right keys [1]: [d_date_sk#136] Join type: Inner Join condition: None -(138) Project [codegen id : 33] +(140) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135, d_date_sk#136, d_year#137] -(139) ReusedExchange [Reuses operator id: 50] +(141) ReusedExchange [Reuses operator id: 50] Output [2]: [d_date_sk#138, d_year#139] -(140) BroadcastHashJoin [codegen id : 33] +(142) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_first_shipto_date_sk#134] Right keys [1]: [d_date_sk#138] Join type: Inner Join condition: None -(141) Project [codegen id : 33] +(143) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137, d_date_sk#138, d_year#139] -(142) ReusedExchange [Reuses operator id: 59] +(144) ReusedExchange [Reuses operator id: 59] Output [2]: [cd_demo_sk#140, cd_marital_status#141] -(143) BroadcastHashJoin [codegen id : 33] +(145) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_cdemo_sk#103] Right keys [1]: [cd_demo_sk#140] Join type: Inner Join condition: None -(144) Project [codegen id : 33] +(146) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_demo_sk#140, cd_marital_status#141] -(145) ReusedExchange [Reuses operator id: 59] +(147) ReusedExchange [Reuses operator id: 59] Output [2]: [cd_demo_sk#142, cd_marital_status#143] -(146) BroadcastHashJoin [codegen id : 33] +(148) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_cdemo_sk#131] Right keys [1]: [cd_demo_sk#142] Join type: Inner Join condition: NOT (cd_marital_status#141 = cd_marital_status#143) -(147) Project [codegen id : 33] +(149) Project [codegen id : 33] Output [14]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [18]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141, cd_demo_sk#142, cd_marital_status#143] -(148) ReusedExchange [Reuses operator id: 68] +(150) ReusedExchange [Reuses operator id: 68] Output [1]: [p_promo_sk#144] -(149) BroadcastHashJoin [codegen id : 33] +(151) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_promo_sk#107] Right keys [1]: [p_promo_sk#144] Join type: Inner Join condition: None -(150) Project [codegen id : 33] +(152) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, p_promo_sk#144] -(151) ReusedExchange [Reuses operator id: 74] +(153) ReusedExchange [Reuses operator id: 74] Output [2]: [hd_demo_sk#145, hd_income_band_sk#146] -(152) BroadcastHashJoin [codegen id : 33] +(154) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_hdemo_sk#104] Right keys [1]: [hd_demo_sk#145] Join type: Inner Join condition: None -(153) Project [codegen id : 33] +(155) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146] Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_demo_sk#145, hd_income_band_sk#146] -(154) ReusedExchange [Reuses operator id: 74] +(156) ReusedExchange [Reuses operator id: 74] Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] -(155) BroadcastHashJoin [codegen id : 33] +(157) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_hdemo_sk#132] Right keys [1]: [hd_demo_sk#147] Join type: Inner Join condition: None -(156) Project [codegen id : 33] +(158) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148] Input [15]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_demo_sk#147, hd_income_band_sk#148] -(157) ReusedExchange [Reuses operator id: 83] +(159) ReusedExchange [Reuses operator id: 83] Output [5]: [ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -(158) BroadcastHashJoin [codegen id : 33] +(160) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_addr_sk#105] Right keys [1]: [ca_address_sk#149] Join type: Inner Join condition: None -(159) Project [codegen id : 33] +(161) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] Input [18]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -(160) ReusedExchange [Reuses operator id: 83] +(162) ReusedExchange [Reuses operator id: 83] Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(161) BroadcastHashJoin [codegen id : 33] +(163) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_addr_sk#133] Right keys [1]: [ca_address_sk#154] Join type: Inner Join condition: None -(162) Project [codegen id : 33] +(164) Project [codegen id : 33] Output [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [21]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(163) ReusedExchange [Reuses operator id: 92] +(165) ReusedExchange [Reuses operator id: 92] Output [1]: [ib_income_band_sk#159] -(164) BroadcastHashJoin [codegen id : 33] +(166) BroadcastHashJoin [codegen id : 33] Left keys [1]: [hd_income_band_sk#146] Right keys [1]: [ib_income_band_sk#159] Join type: Inner Join condition: None -(165) Project [codegen id : 33] +(167) Project [codegen id : 33] Output [18]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [20]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#159] -(166) ReusedExchange [Reuses operator id: 92] +(168) ReusedExchange [Reuses operator id: 92] Output [1]: [ib_income_band_sk#160] -(167) BroadcastHashJoin [codegen id : 33] +(169) BroadcastHashJoin [codegen id : 33] Left keys [1]: [hd_income_band_sk#148] Right keys [1]: [ib_income_band_sk#160] Join type: Inner Join condition: None -(168) Project [codegen id : 33] +(170) Project [codegen id : 33] Output [17]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#160] -(169) ReusedExchange [Reuses operator id: 102] +(171) ReusedExchange [Reuses operator id: 102] Output [2]: [i_item_sk#161, i_product_name#162] -(170) BroadcastHashJoin [codegen id : 33] +(172) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_item_sk#101] Right keys [1]: [i_item_sk#161] Join type: Inner Join condition: None -(171) Project [codegen id : 33] +(173) Project [codegen id : 33] Output [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -(172) HashAggregate [codegen id : 33] +(174) HashAggregate [codegen id : 33] Input [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#109)), partial_sum(UnscaledValue(ss_list_price#110)), partial_sum(UnscaledValue(ss_coupon_amt#111))] Aggregate Attributes [4]: [count#72, sum#163, sum#164, sum#165] Results [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] -(173) HashAggregate [codegen id : 33] +(175) HashAggregate [codegen id : 33] Input [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#109)), sum(UnscaledValue(ss_list_price#110)), sum(UnscaledValue(ss_coupon_amt#111))] Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#109))#81, sum(UnscaledValue(ss_list_price#110))#82, sum(UnscaledValue(ss_coupon_amt#111))#83] Results [8]: [i_item_sk#161 AS item_sk#169, s_store_name#128 AS store_name#170, s_zip#129 AS store_zip#171, d_year#126 AS syear#172, count(1)#80 AS cnt#173, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#109))#81,17,2) AS s1#174, MakeDecimal(sum(UnscaledValue(ss_list_price#110))#82,17,2) AS s2#175, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#111))#83,17,2) AS s3#176] -(174) Exchange +(176) RowToColumnar +Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] + +(177) CometColumnarExchange +Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] +Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] + +(178) CometSort Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176], [item_sk#169 ASC NULLS FIRST, store_name#170 ASC NULLS FIRST, store_zip#171 ASC NULLS FIRST] -(175) Sort [codegen id : 34] +(179) ColumnarToRow [codegen id : 34] Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: [item_sk#169 ASC NULLS FIRST, store_name#170 ASC NULLS FIRST, store_zip#171 ASC NULLS FIRST], false, 0 -(176) SortMergeJoin [codegen id : 35] +(180) SortMergeJoin [codegen id : 35] Left keys [3]: [item_sk#85, store_name#86, store_zip#87] Right keys [3]: [item_sk#169, store_name#170, store_zip#171] Join type: Inner Join condition: (cnt#173 <= cnt#97) -(177) Project [codegen id : 35] +(181) Project [codegen id : 35] Output [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] Input [25]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -(178) Exchange +(182) RowToColumnar +Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] + +(183) CometColumnarExchange +Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] +Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] + +(184) CometSort Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] +Arguments: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173], [product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST] -(179) Sort [codegen id : 36] +(185) ColumnarToRow [codegen id : 36] Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: [product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (183) -+- * ColumnarToRow (182) - +- CometFilter (181) - +- CometScan parquet spark_catalog.default.date_dim (180) +BroadcastExchange (189) ++- * ColumnarToRow (188) + +- CometFilter (187) + +- CometScan parquet spark_catalog.default.date_dim (186) (unknown) Scan parquet spark_catalog.default.date_dim @@ -1008,22 +1032,22 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(181) CometFilter +(187) CometFilter Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(182) ColumnarToRow [codegen id : 1] +(188) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] -(183) BroadcastExchange +(189) BroadcastExchange Input [2]: [d_date_sk#32, d_year#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 109 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 -BroadcastExchange (187) -+- * ColumnarToRow (186) - +- CometFilter (185) - +- CometScan parquet spark_catalog.default.date_dim (184) +Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 +BroadcastExchange (193) ++- * ColumnarToRow (192) + +- CometFilter (191) + +- CometScan parquet spark_catalog.default.date_dim (190) (unknown) Scan parquet spark_catalog.default.date_dim @@ -1033,14 +1057,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(185) CometFilter +(191) CometFilter Input [2]: [d_date_sk#125, d_year#126] Condition : ((isnotnull(d_year#126) AND (d_year#126 = 2000)) AND isnotnull(d_date_sk#125)) -(186) ColumnarToRow [codegen id : 1] +(192) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#125, d_year#126] -(187) BroadcastExchange +(193) BroadcastExchange Input [2]: [d_date_sk#125, d_year#126] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index eda7db4e57..9fb039a96d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -1,254 +1,260 @@ WholeStageCodegen (36) - Sort [product_name,store_name,cnt] + ColumnarToRow InputAdapter - Exchange [product_name,store_name,cnt] #1 - WholeStageCodegen (35) - Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] - InputAdapter - WholeStageCodegen (17) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (16) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometExchange [ss_item_sk] #3 + CometSort [product_name,store_name,cnt] + CometColumnarExchange [product_name,store_name,cnt] #1 + RowToColumnar + WholeStageCodegen (35) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (17) + ColumnarToRow + InputAdapter + CometSort [item_sk,store_name,store_zip] + CometColumnarExchange [item_sk,store_name,store_zip] #2 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometSortMergeJoin [ss_item_sk,cs_item_sk] + CometSort [ss_item_sk] + CometColumnarExchange [ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #4 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [sale,refund] + CometHashAggregate [cs_item_sk,sum,sum,isEmpty] + CometColumnarExchange [cs_item_sk] #6 + CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number] + CometColumnarExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + BroadcastExchange #10 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) + BroadcastExchange #11 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (4) + BroadcastExchange #12 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (6) + BroadcastExchange #13 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + BroadcastExchange #14 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - BroadcastExchange #14 - WholeStageCodegen (9) + BroadcastExchange #15 + WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) + BroadcastExchange #16 + WholeStageCodegen (13) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (13) + BroadcastExchange #17 + WholeStageCodegen (15) ColumnarToRow InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (15) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - InputAdapter - WholeStageCodegen (34) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (33) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometExchange [ss_item_sk] #19 + CometProject [i_item_sk,i_product_name] + CometFilter [i_current_price,i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (34) + ColumnarToRow + InputAdapter + CometSort [item_sk,store_name,store_zip] + CometColumnarExchange [item_sk,store_name,store_zip] #18 + RowToColumnar + WholeStageCodegen (33) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + CometSortMergeJoin [ss_item_sk,cs_item_sk] + CometSort [ss_item_sk] + CometColumnarExchange [ss_item_sk] #19 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #20 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [sale,refund] + CometHashAggregate [cs_item_sk,sum,sum,isEmpty] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #21 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 InputAdapter - ReusedExchange [d_date_sk,d_year] #21 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [p_promo_sk] #13 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [p_promo_sk] #13 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + ReusedExchange [i_item_sk,i_product_name] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 1a06f27fb9..3ef59799a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -1,43 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) +TakeOrderedAndProject (43) ++- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) : : :- * ColumnarToRow (3) : : : +- CometFilter (2) : : : +- CometScan parquet spark_catalog.default.store (1) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * HashAggregate (12) - : : +- Exchange (11) - : : +- * HashAggregate (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : +- ReusedExchange (7) - : +- BroadcastExchange (20) - : +- * ColumnarToRow (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - +- BroadcastExchange (36) - +- * Filter (35) - +- * HashAggregate (34) - +- Exchange (33) - +- * HashAggregate (32) - +- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * ColumnarToRow (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.store_sales (23) - +- ReusedExchange (26) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometHashAggregate (13) + : : +- CometColumnarExchange (12) + : : +- RowToColumnar (11) + : : +- * HashAggregate (10) + : : +- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : +- ReusedExchange (7) + : +- BroadcastExchange (22) + : +- * ColumnarToRow (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- BroadcastExchange (40) + +- * ColumnarToRow (39) + +- CometFilter (38) + +- CometHashAggregate (37) + +- CometColumnarExchange (36) + +- CometHashAggregate (35) + +- CometHashAggregate (34) + +- CometColumnarExchange (33) + +- RowToColumnar (32) + +- * HashAggregate (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.store_sales (25) + +- ReusedExchange (28) (unknown) Scan parquet spark_catalog.default.store @@ -51,7 +55,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) ColumnarToRow [codegen id : 9] +(3) ColumnarToRow [codegen id : 8] Input [2]: [s_store_sk#1, s_store_name#2] (unknown) Scan parquet spark_catalog.default.store_sales @@ -69,7 +73,7 @@ Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) (6) ColumnarToRow [codegen id : 2] Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -(7) ReusedExchange [Reuses operator id: 44] +(7) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#8] (8) BroadcastHashJoin [codegen id : 2] @@ -89,181 +93,185 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] Aggregate Attributes [1]: [sum#9] Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -(11) Exchange +(11) RowToColumnar Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(12) HashAggregate [codegen id : 3] +(12) CometColumnarExchange +Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] +Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(13) CometHashAggregate Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] -Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] -(13) Filter [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Condition : isnotnull(revenue#12) +(14) CometFilter +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Condition : isnotnull(revenue#11) -(14) BroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] +(15) ColumnarToRow [codegen id : 3] +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] + +(16) BroadcastExchange +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#1] Right keys [1]: [ss_store_sk#4] Join type: Inner Join condition: None -(16) Project [codegen id : 9] -Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] +(18) Project [codegen id : 8] +Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] +Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] (unknown) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(18) CometFilter -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Condition : isnotnull(i_item_sk#13) +(20) CometFilter +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Condition : isnotnull(i_item_sk#12) -(19) ColumnarToRow [codegen id : 4] -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +(21) ColumnarToRow [codegen id : 4] +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -(20) BroadcastExchange -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +(22) BroadcastExchange +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(21) BroadcastHashJoin [codegen id : 9] +(23) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_item_sk#3] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#12] Join type: Inner Join condition: None -(22) Project [codegen id : 9] -Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +(24) Project [codegen id : 8] +Output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +Output [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(24) CometFilter -Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_store_sk#19) +(26) CometFilter +Input [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] +Condition : isnotnull(ss_store_sk#18) -(25) ColumnarToRow [codegen id : 6] -Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +(27) ColumnarToRow [codegen id : 6] +Input [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] -(26) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#23] +(28) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#22] -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#21] -Right keys [1]: [d_date_sk#23] +(29) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#20] +Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(28) Project [codegen id : 6] -Output [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] - -(29) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] - -(30) Exchange -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] -Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#20))#26] -Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#26,17,2) AS revenue#27] - -(32) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#19, revenue#27] -Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#27)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ss_store_sk#19, sum#30, count#31] - -(33) Exchange -Input [3]: [ss_store_sk#19, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(34) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#19, sum#30, count#31] -Keys [1]: [ss_store_sk#19] -Functions [1]: [avg(revenue#27)] -Aggregate Attributes [1]: [avg(revenue#27)#32] -Results [2]: [ss_store_sk#19, avg(revenue#27)#32 AS ave#33] - -(35) Filter [codegen id : 8] -Input [2]: [ss_store_sk#19, ave#33] -Condition : isnotnull(ave#33) - -(36) BroadcastExchange -Input [2]: [ss_store_sk#19, ave#33] +(30) Project [codegen id : 6] +Output [3]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19] +Input [5]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20, d_date_sk#22] + +(31) HashAggregate [codegen id : 6] +Input [3]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19] +Keys [2]: [ss_store_sk#18, ss_item_sk#17] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#19))] +Aggregate Attributes [1]: [sum#23] +Results [3]: [ss_store_sk#18, ss_item_sk#17, sum#24] + +(32) RowToColumnar +Input [3]: [ss_store_sk#18, ss_item_sk#17, sum#24] + +(33) CometColumnarExchange +Input [3]: [ss_store_sk#18, ss_item_sk#17, sum#24] +Arguments: hashpartitioning(ss_store_sk#18, ss_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometHashAggregate +Input [3]: [ss_store_sk#18, ss_item_sk#17, sum#24] +Keys [2]: [ss_store_sk#18, ss_item_sk#17] +Functions [1]: [sum(UnscaledValue(ss_sales_price#19))] + +(35) CometHashAggregate +Input [2]: [ss_store_sk#18, revenue#25] +Keys [1]: [ss_store_sk#18] +Functions [1]: [partial_avg(revenue#25)] + +(36) CometColumnarExchange +Input [3]: [ss_store_sk#18, sum#26, count#27] +Arguments: hashpartitioning(ss_store_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(37) CometHashAggregate +Input [3]: [ss_store_sk#18, sum#26, count#27] +Keys [1]: [ss_store_sk#18] +Functions [1]: [avg(revenue#25)] + +(38) CometFilter +Input [2]: [ss_store_sk#18, ave#28] +Condition : isnotnull(ave#28) + +(39) ColumnarToRow [codegen id : 7] +Input [2]: [ss_store_sk#18, ave#28] + +(40) BroadcastExchange +Input [2]: [ss_store_sk#18, ave#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#19] +Right keys [1]: [ss_store_sk#18] Join type: Inner -Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) +Join condition: (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#28)) -(38) Project [codegen id : 9] -Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17, ss_store_sk#19, ave#33] +(42) Project [codegen id : 8] +Output [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16, ss_store_sk#18, ave#28] -(39) TakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] +(43) TakeOrderedAndProject +Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#34] +Output [2]: [d_date_sk#8, d_month_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_month_seq#34 <= 1187)) AND isnotnull(d_date_sk#8)) +(45) CometFilter +Input [2]: [d_date_sk#8, d_month_seq#29] +Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1176)) AND (d_month_seq#29 <= 1187)) AND isnotnull(d_date_sk#8)) -(42) CometProject -Input [2]: [d_date_sk#8, d_month_seq#34] +(46) CometProject +Input [2]: [d_date_sk#8, d_month_seq#29] Arguments: [d_date_sk#8], [d_date_sk#8] -(43) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(44) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index 33b695e811..5a65234a30 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (9) + WholeStageCodegen (8) Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] @@ -13,28 +13,30 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes InputAdapter BroadcastExchange #1 WholeStageCodegen (3) - Filter [revenue] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - InputAdapter - Exchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + CometFilter [revenue] + CometHashAggregate [ss_store_sk,ss_item_sk,sum] + CometColumnarExchange [ss_store_sk,ss_item_sk] #2 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 WholeStageCodegen (4) @@ -44,16 +46,16 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #5 - WholeStageCodegen (8) - Filter [ave] - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - InputAdapter - Exchange [ss_store_sk] #6 - WholeStageCodegen (7) - HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - InputAdapter - Exchange [ss_store_sk,ss_item_sk] #7 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [ave] + CometHashAggregate [ss_store_sk,sum,count] + CometColumnarExchange [ss_store_sk] #6 + CometHashAggregate [ss_store_sk,revenue] + CometHashAggregate [ss_store_sk,ss_item_sk,sum] + CometColumnarExchange [ss_store_sk,ss_item_sk] #7 + RowToColumnar WholeStageCodegen (6) HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] Project [ss_item_sk,ss_store_sk,ss_sales_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 41e783d134..f4614e57bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -1,56 +1,59 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- Union (48) - :- * HashAggregate (29) - : +- Exchange (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.time_dim (13) - : +- BroadcastExchange (24) - : +- * ColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.ship_mode (20) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (32) - : : : : +- CometFilter (31) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- ReusedExchange (42) +* ColumnarToRow (55) ++- CometTakeOrderedAndProject (54) + +- CometHashAggregate (53) + +- CometColumnarExchange (52) + +- CometHashAggregate (51) + +- CometUnion (50) + :- CometHashAggregate (30) + : +- CometColumnarExchange (29) + : +- RowToColumnar (28) + : +- * HashAggregate (27) + : +- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.warehouse (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (17) + : : +- * ColumnarToRow (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.time_dim (13) + : +- BroadcastExchange (24) + : +- * ColumnarToRow (23) + : +- CometProject (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.ship_mode (20) + +- CometHashAggregate (49) + +- CometColumnarExchange (48) + +- RowToColumnar (47) + +- * HashAggregate (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * Project (36) + : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : :- * ColumnarToRow (33) + : : : : +- CometFilter (32) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : : +- ReusedExchange (34) + : : +- ReusedExchange (37) + : +- ReusedExchange (40) + +- ReusedExchange (43) (unknown) Scan parquet spark_catalog.default.web_sales @@ -96,7 +99,7 @@ Join condition: None Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -(10) ReusedExchange [Reuses operator id: 56] +(10) ReusedExchange [Reuses operator id: 59] Output [3]: [d_date_sk#16, d_year#17, d_moy#18] (11) BroadcastHashJoin [codegen id : 5] @@ -180,133 +183,136 @@ Functions [24]: [partial_sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 Aggregate Attributes [48]: [sum#23, isEmpty#24, sum#25, isEmpty#26, sum#27, isEmpty#28, sum#29, isEmpty#30, sum#31, isEmpty#32, sum#33, isEmpty#34, sum#35, isEmpty#36, sum#37, isEmpty#38, sum#39, isEmpty#40, sum#41, isEmpty#42, sum#43, isEmpty#44, sum#45, isEmpty#46, sum#47, isEmpty#48, sum#49, isEmpty#50, sum#51, isEmpty#52, sum#53, isEmpty#54, sum#55, isEmpty#56, sum#57, isEmpty#58, sum#59, isEmpty#60, sum#61, isEmpty#62, sum#63, isEmpty#64, sum#65, isEmpty#66, sum#67, isEmpty#68, sum#69, isEmpty#70] Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] -(28) Exchange +(28) RowToColumnar Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) HashAggregate [codegen id : 6] +(29) CometColumnarExchange +Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometHashAggregate Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17] Functions [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142] -Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#143, d_year#17 AS year#144, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119 AS jan_sales#145, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120 AS feb_sales#146, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121 AS mar_sales#147, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122 AS apr_sales#148, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123 AS may_sales#149, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124 AS jun_sales#150, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125 AS jul_sales#151, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126 AS aug_sales#152, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127 AS sep_sales#153, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128 AS oct_sales#154, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129 AS nov_sales#155, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130 AS dec_sales#156, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131 AS jan_net#157, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132 AS feb_net#158, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133 AS mar_net#159, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134 AS apr_net#160, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135 AS may_net#161, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136 AS jun_net#162, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137 AS jul_net#163, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138 AS aug_net#164, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139 AS sep_net#165, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140 AS oct_net#166, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141 AS nov_net#167, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142 AS dec_net#168] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] +Output [7]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_warehouse_sk#121, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#175), dynamicpruningexpression(cs_sold_date_sk#175 IN dynamicpruning#176)] +PartitionFilters: [isnotnull(cs_sold_date_sk#125), dynamicpruningexpression(cs_sold_date_sk#125 IN dynamicpruning#126)] PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(31) CometFilter -Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] -Condition : ((isnotnull(cs_warehouse_sk#171) AND isnotnull(cs_sold_time_sk#169)) AND isnotnull(cs_ship_mode_sk#170)) +(32) CometFilter +Input [7]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_warehouse_sk#121, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125] +Condition : ((isnotnull(cs_warehouse_sk#121) AND isnotnull(cs_sold_time_sk#119)) AND isnotnull(cs_ship_mode_sk#120)) -(32) ColumnarToRow [codegen id : 11] -Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] +(33) ColumnarToRow [codegen id : 10] +Input [7]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_warehouse_sk#121, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125] -(33) ReusedExchange [Reuses operator id: 7] -Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] +(34) ReusedExchange [Reuses operator id: 7] +Output [7]: [w_warehouse_sk#127, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133] -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_warehouse_sk#171] -Right keys [1]: [w_warehouse_sk#177] +(35) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_warehouse_sk#121] +Right keys [1]: [w_warehouse_sk#127] Join type: Inner Join condition: None -(35) Project [codegen id : 11] -Output [12]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] +(36) Project [codegen id : 10] +Output [12]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133] +Input [14]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_warehouse_sk#121, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125, w_warehouse_sk#127, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133] -(36) ReusedExchange [Reuses operator id: 56] -Output [3]: [d_date_sk#184, d_year#185, d_moy#186] +(37) ReusedExchange [Reuses operator id: 59] +Output [3]: [d_date_sk#134, d_year#135, d_moy#136] -(37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#175] -Right keys [1]: [d_date_sk#184] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#125] +Right keys [1]: [d_date_sk#134] Join type: Inner Join condition: None -(38) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [15]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] +(39) Project [codegen id : 10] +Output [13]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136] +Input [15]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_date_sk#134, d_year#135, d_moy#136] -(39) ReusedExchange [Reuses operator id: 17] -Output [1]: [t_time_sk#187] +(40) ReusedExchange [Reuses operator id: 17] +Output [1]: [t_time_sk#137] -(40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_time_sk#169] -Right keys [1]: [t_time_sk#187] +(41) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_time_sk#119] +Right keys [1]: [t_time_sk#137] Join type: Inner Join condition: None -(41) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] +(42) Project [codegen id : 10] +Output [12]: [cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136] +Input [14]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136, t_time_sk#137] -(42) ReusedExchange [Reuses operator id: 24] -Output [1]: [sm_ship_mode_sk#188] +(43) ReusedExchange [Reuses operator id: 24] +Output [1]: [sm_ship_mode_sk#138] -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_ship_mode_sk#170] -Right keys [1]: [sm_ship_mode_sk#188] +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_ship_mode_sk#120] +Right keys [1]: [sm_ship_mode_sk#138] Join type: Inner Join condition: None -(44) Project [codegen id : 11] -Output [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [13]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] - -(45) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] -Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236] -Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] - -(46) Exchange -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(47) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] -Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308] -Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, DHL,BARIAN AS ship_carriers#309, d_year#185 AS year#310, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285 AS jan_sales#311, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286 AS feb_sales#312, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287 AS mar_sales#313, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288 AS apr_sales#314, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289 AS may_sales#315, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290 AS jun_sales#316, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291 AS jul_sales#317, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292 AS aug_sales#318, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293 AS sep_sales#319, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294 AS oct_sales#320, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295 AS nov_sales#321, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296 AS dec_sales#322, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297 AS jan_net#323, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298 AS feb_net#324, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299 AS mar_net#325, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300 AS apr_net#326, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301 AS may_net#327, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302 AS jun_net#328, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303 AS jul_net#329, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304 AS aug_net#330, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305 AS sep_net#331, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306 AS oct_net#332, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307 AS nov_net#333, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308 AS dec_net#334] - -(48) Union - -(49) HashAggregate [codegen id : 13] -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#145, feb_sales#146, mar_sales#147, apr_sales#148, may_sales#149, jun_sales#150, jul_sales#151, aug_sales#152, sep_sales#153, oct_sales#154, nov_sales#155, dec_sales#156, jan_net#157, feb_net#158, mar_net#159, apr_net#160, may_net#161, jun_net#162, jul_net#163, aug_net#164, sep_net#165, oct_net#166, nov_net#167, dec_net#168] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] -Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial_sum(mar_sales#147), partial_sum(apr_sales#148), partial_sum(may_sales#149), partial_sum(jun_sales#150), partial_sum(jul_sales#151), partial_sum(aug_sales#152), partial_sum(sep_sales#153), partial_sum(oct_sales#154), partial_sum(nov_sales#155), partial_sum(dec_sales#156), partial_sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#157), partial_sum(feb_net#158), partial_sum(mar_net#159), partial_sum(apr_net#160), partial_sum(may_net#161), partial_sum(jun_net#162), partial_sum(jul_net#163), partial_sum(aug_net#164), partial_sum(sep_net#165), partial_sum(oct_net#166), partial_sum(nov_net#167), partial_sum(dec_net#168)] -Aggregate Attributes [72]: [sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384, sum#385, isEmpty#386, sum#387, isEmpty#388, sum#389, isEmpty#390, sum#391, isEmpty#392, sum#393, isEmpty#394, sum#395, isEmpty#396, sum#397, isEmpty#398, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406] -Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] - -(50) Exchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(51) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] -Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#157), sum(feb_net#158), sum(mar_net#159), sum(apr_net#160), sum(may_net#161), sum(jun_net#162), sum(jul_net#163), sum(aug_net#164), sum(sep_net#165), sum(oct_net#166), sum(nov_net#167), sum(dec_net#168)] -Aggregate Attributes [36]: [sum(jan_sales#145)#479, sum(feb_sales#146)#480, sum(mar_sales#147)#481, sum(apr_sales#148)#482, sum(may_sales#149)#483, sum(jun_sales#150)#484, sum(jul_sales#151)#485, sum(aug_sales#152)#486, sum(sep_sales#153)#487, sum(oct_sales#154)#488, sum(nov_sales#155)#489, sum(dec_sales#156)#490, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502, sum(jan_net#157)#503, sum(feb_net#158)#504, sum(mar_net#159)#505, sum(apr_net#160)#506, sum(may_net#161)#507, sum(jun_net#162)#508, sum(jul_net#163)#509, sum(aug_net#164)#510, sum(sep_net#165)#511, sum(oct_net#166)#512, sum(nov_net#167)#513, sum(dec_net#168)#514] -Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum(jan_sales#145)#479 AS jan_sales#515, sum(feb_sales#146)#480 AS feb_sales#516, sum(mar_sales#147)#481 AS mar_sales#517, sum(apr_sales#148)#482 AS apr_sales#518, sum(may_sales#149)#483 AS may_sales#519, sum(jun_sales#150)#484 AS jun_sales#520, sum(jul_sales#151)#485 AS jul_sales#521, sum(aug_sales#152)#486 AS aug_sales#522, sum(sep_sales#153)#487 AS sep_sales#523, sum(oct_sales#154)#488 AS oct_sales#524, sum(nov_sales#155)#489 AS nov_sales#525, sum(dec_sales#156)#490 AS dec_sales#526, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491 AS jan_sales_per_sq_foot#527, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492 AS feb_sales_per_sq_foot#528, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493 AS mar_sales_per_sq_foot#529, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494 AS apr_sales_per_sq_foot#530, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495 AS may_sales_per_sq_foot#531, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496 AS jun_sales_per_sq_foot#532, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497 AS jul_sales_per_sq_foot#533, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498 AS aug_sales_per_sq_foot#534, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499 AS sep_sales_per_sq_foot#535, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500 AS oct_sales_per_sq_foot#536, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501 AS nov_sales_per_sq_foot#537, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502 AS dec_sales_per_sq_foot#538, sum(jan_net#157)#503 AS jan_net#539, sum(feb_net#158)#504 AS feb_net#540, sum(mar_net#159)#505 AS mar_net#541, sum(apr_net#160)#506 AS apr_net#542, sum(may_net#161)#507 AS may_net#543, sum(jun_net#162)#508 AS jun_net#544, sum(jul_net#163)#509 AS jul_net#545, sum(aug_net#164)#510 AS aug_net#546, sum(sep_net#165)#511 AS sep_net#547, sum(oct_net#166)#512 AS oct_net#548, sum(nov_net#167)#513 AS nov_net#549, sum(dec_net#168)#514 AS dec_net#550] - -(52) TakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] -Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] +(45) Project [codegen id : 10] +Output [11]: [cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136] +Input [13]: [cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136, sm_ship_mode_sk#138] + +(46) HashAggregate [codegen id : 10] +Input [11]: [cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136] +Keys [7]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135] +Functions [24]: [partial_sum(CASE WHEN (d_moy#136 = 1) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 2) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 3) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 4) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 5) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 6) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 7) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 8) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 9) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 10) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 11) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 12) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 1) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 2) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 3) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 4) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 5) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 6) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 7) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 8) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 9) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 10) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 11) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 12) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] +Results [55]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234] + +(47) RowToColumnar +Input [55]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234] + +(48) CometColumnarExchange +Input [55]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234] +Arguments: hashpartitioning(w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(49) CometHashAggregate +Input [55]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234] +Keys [7]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135] +Functions [24]: [sum(CASE WHEN (d_moy#136 = 1) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 2) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 3) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 4) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 5) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 6) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 7) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 8) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 9) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 10) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 11) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 12) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 1) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 2) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 3) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 4) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 5) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 6) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 7) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 8) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 9) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 10) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 11) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 12) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END)] + +(50) CometUnion +Child 0 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, jan_sales#237, feb_sales#238, mar_sales#239, apr_sales#240, may_sales#241, jun_sales#242, jul_sales#243, aug_sales#244, sep_sales#245, oct_sales#246, nov_sales#247, dec_sales#248, jan_net#249, feb_net#250, mar_net#251, apr_net#252, may_net#253, jun_net#254, jul_net#255, aug_net#256, sep_net#257, oct_net#258, nov_net#259, dec_net#260] +Child 1 Input [32]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, ship_carriers#261, year#262, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_net#275, feb_net#276, mar_net#277, apr_net#278, may_net#279, jun_net#280, jul_net#281, aug_net#282, sep_net#283, oct_net#284, nov_net#285, dec_net#286] + +(51) CometHashAggregate +Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, jan_sales#237, feb_sales#238, mar_sales#239, apr_sales#240, may_sales#241, jun_sales#242, jul_sales#243, aug_sales#244, sep_sales#245, oct_sales#246, nov_sales#247, dec_sales#248, jan_net#249, feb_net#250, mar_net#251, apr_net#252, may_net#253, jun_net#254, jul_net#255, aug_net#256, sep_net#257, oct_net#258, nov_net#259, dec_net#260] +Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236] +Functions [36]: [partial_sum(jan_sales#237), partial_sum(feb_sales#238), partial_sum(mar_sales#239), partial_sum(apr_sales#240), partial_sum(may_sales#241), partial_sum(jun_sales#242), partial_sum(jul_sales#243), partial_sum(aug_sales#244), partial_sum(sep_sales#245), partial_sum(oct_sales#246), partial_sum(nov_sales#247), partial_sum(dec_sales#248), partial_sum((jan_sales#237 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#238 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#239 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#240 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#241 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#242 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#243 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#244 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#245 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#246 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#247 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#248 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#249), partial_sum(feb_net#250), partial_sum(mar_net#251), partial_sum(apr_net#252), partial_sum(may_net#253), partial_sum(jun_net#254), partial_sum(jul_net#255), partial_sum(aug_net#256), partial_sum(sep_net#257), partial_sum(oct_net#258), partial_sum(nov_net#259), partial_sum(dec_net#260)] + +(52) CometColumnarExchange +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, sum#287, isEmpty#288, sum#289, isEmpty#290, sum#291, isEmpty#292, sum#293, isEmpty#294, sum#295, isEmpty#296, sum#297, isEmpty#298, sum#299, isEmpty#300, sum#301, isEmpty#302, sum#303, isEmpty#304, sum#305, isEmpty#306, sum#307, isEmpty#308, sum#309, isEmpty#310, sum#311, isEmpty#312, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(53) CometHashAggregate +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, sum#287, isEmpty#288, sum#289, isEmpty#290, sum#291, isEmpty#292, sum#293, isEmpty#294, sum#295, isEmpty#296, sum#297, isEmpty#298, sum#299, isEmpty#300, sum#301, isEmpty#302, sum#303, isEmpty#304, sum#305, isEmpty#306, sum#307, isEmpty#308, sum#309, isEmpty#310, sum#311, isEmpty#312, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358] +Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236] +Functions [36]: [sum(jan_sales#237), sum(feb_sales#238), sum(mar_sales#239), sum(apr_sales#240), sum(may_sales#241), sum(jun_sales#242), sum(jul_sales#243), sum(aug_sales#244), sum(sep_sales#245), sum(oct_sales#246), sum(nov_sales#247), sum(dec_sales#248), sum((jan_sales#237 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#238 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#239 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#240 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#241 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#242 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#243 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#244 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#245 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#246 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#247 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#248 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#249), sum(feb_net#250), sum(mar_net#251), sum(apr_net#252), sum(may_net#253), sum(jun_net#254), sum(jul_net#255), sum(aug_net#256), sum(sep_net#257), sum(oct_net#258), sum(nov_net#259), sum(dec_net#260)] + +(54) CometTakeOrderedAndProject +Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, jan_sales#359, feb_sales#360, mar_sales#361, apr_sales#362, may_sales#363, jun_sales#364, jul_sales#365, aug_sales#366, sep_sales#367, oct_sales#368, nov_sales#369, dec_sales#370, jan_sales_per_sq_foot#371, feb_sales_per_sq_foot#372, mar_sales_per_sq_foot#373, apr_sales_per_sq_foot#374, may_sales_per_sq_foot#375, jun_sales_per_sq_foot#376, jul_sales_per_sq_foot#377, aug_sales_per_sq_foot#378, sep_sales_per_sq_foot#379, oct_sales_per_sq_foot#380, nov_sales_per_sq_foot#381, dec_sales_per_sq_foot#382, jan_net#383, feb_net#384, mar_net#385, apr_net#386, may_net#387, jun_net#388, jul_net#389, aug_net#390, sep_net#391, oct_net#392, nov_net#393, dec_net#394] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#14,w_country#15,ship_carriers#235,year#236,jan_sales#359,feb_sales#360,mar_sales#361,apr_sales#362,may_sales#363,jun_sales#364,jul_sales#365,aug_sales#366,sep_sales#367,oct_sales#368,nov_sales#369,dec_sales#370,jan_sales_per_sq_foot#371,feb_sales_per_sq_foot#372,mar_sales_per_sq_foot#373,apr_sales_per_sq_foot#374,may_sales_per_sq_foot#375,jun_sales_per_sq_foot#376,jul_sales_per_sq_foot#377,aug_sales_per_sq_foot#378,sep_sales_per_sq_foot#379,oct_sales_per_sq_foot#380,nov_sales_per_sq_foot#381,dec_sales_per_sq_foot#382,jan_net#383,feb_net#384,mar_net#385,apr_net#386,may_net#387,jun_net#388,jul_net#389,aug_net#390,sep_net#391,oct_net#392,nov_net#393,dec_net#394]), 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, jan_sales#359, feb_sales#360, mar_sales#361, apr_sales#362, may_sales#363, jun_sales#364, jul_sales#365, aug_sales#366, sep_sales#367, oct_sales#368, nov_sales#369, dec_sales#370, jan_sales_per_sq_foot#371, feb_sales_per_sq_foot#372, mar_sales_per_sq_foot#373, apr_sales_per_sq_foot#374, may_sales_per_sq_foot#375, jun_sales_per_sq_foot#376, jul_sales_per_sq_foot#377, aug_sales_per_sq_foot#378, sep_sales_per_sq_foot#379, oct_sales_per_sq_foot#380, nov_sales_per_sq_foot#381, dec_sales_per_sq_foot#382, jan_net#383, feb_net#384, mar_net#385, apr_net#386, may_net#387, jun_net#388, jul_net#389, aug_net#390, sep_net#391, oct_net#392, nov_net#393, dec_net#394] + +(55) ColumnarToRow [codegen id : 11] +Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, jan_sales#359, feb_sales#360, mar_sales#361, apr_sales#362, may_sales#363, jun_sales#364, jul_sales#365, aug_sales#366, sep_sales#367, oct_sales#368, nov_sales#369, dec_sales#370, jan_sales_per_sq_foot#371, feb_sales_per_sq_foot#372, mar_sales_per_sq_foot#373, apr_sales_per_sq_foot#374, may_sales_per_sq_foot#375, jun_sales_per_sq_foot#376, jul_sales_per_sq_foot#377, aug_sales_per_sq_foot#378, sep_sales_per_sq_foot#379, oct_sales_per_sq_foot#380, nov_sales_per_sq_foot#381, dec_sales_per_sq_foot#382, jan_net#383, feb_net#384, mar_net#385, apr_net#386, may_net#387, jun_net#388, jul_net#389, aug_net#390, sep_net#391, oct_net#392, nov_net#393, dec_net#394] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (56) -+- * ColumnarToRow (55) - +- CometFilter (54) - +- CometScan parquet spark_catalog.default.date_dim (53) +BroadcastExchange (59) ++- * ColumnarToRow (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) (unknown) Scan parquet spark_catalog.default.date_dim @@ -316,17 +322,17 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(54) CometFilter +(57) CometFilter Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(55) ColumnarToRow [codegen id : 1] +(58) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -(56) BroadcastExchange +(59) BroadcastExchange Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#175 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#125 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index 8ed74582f0..025a2b9d4d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -1,86 +1,83 @@ -TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] +WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] + CometUnion + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + ReusedExchange [d_date_sk,d_year,d_moy] #3 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometProject [t_time_sk] + CometFilter [t_time,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [sm_ship_mode_sk] + CometFilter [sm_carrier,sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_sold_time_sk,t_time_sk] + Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 + ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 InputAdapter - ReusedExchange [t_time_sk] #5 + ReusedExchange [d_date_sk,d_year,d_moy] #3 InputAdapter - ReusedExchange [sm_ship_mode_sk] #6 + ReusedExchange [t_time_sk] #5 + InputAdapter + ReusedExchange [sm_ship_mode_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index 5dabc82d2c..fc5ae3bc19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -1,31 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (27) -+- * Filter (26) - +- Window (25) - +- * Sort (24) - +- Exchange (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.item (13) +TakeOrderedAndProject (29) ++- * Filter (28) + +- Window (27) + +- * ColumnarToRow (26) + +- CometSort (25) + +- CometColumnarExchange (24) + +- CometHashAggregate (23) + +- CometColumnarExchange (22) + +- RowToColumnar (21) + +- * HashAggregate (20) + +- * Expand (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.store (7) + +- BroadcastExchange (16) + +- * ColumnarToRow (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.item (13) (unknown) Scan parquet spark_catalog.default.store_sales @@ -43,7 +45,7 @@ Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 32] +(4) ReusedExchange [Reuses operator id: 34] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (5) BroadcastHashJoin [codegen id : 4] @@ -123,66 +125,70 @@ Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as d Aggregate Attributes [2]: [sum#27, isEmpty#28] Results [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] -(21) Exchange +(21) RowToColumnar Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] -Arguments: hashpartitioning(i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) HashAggregate [codegen id : 5] +(22) CometColumnarExchange +Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] +Arguments: hashpartitioning(i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(23) CometHashAggregate Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31] -Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31 AS sumsales#32] -(23) Exchange -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(24) CometColumnarExchange +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31] +Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(25) CometSort +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31] +Arguments: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31], [i_category#18 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST] -(24) Sort [codegen id : 6] -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0 +(26) ColumnarToRow [codegen id : 5] +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31] -(25) Window -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: [rank(sumsales#32) windowspecdefinition(i_category#18, sumsales#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#33], [i_category#18], [sumsales#32 DESC NULLS LAST] +(27) Window +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31] +Arguments: [rank(sumsales#31) windowspecdefinition(i_category#18, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [i_category#18], [sumsales#31 DESC NULLS LAST] -(26) Filter [codegen id : 7] -Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] -Condition : (rk#33 <= 100) +(28) Filter [codegen id : 6] +Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31, rk#32] +Condition : (rk#32 <= 100) -(27) TakeOrderedAndProject -Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] -Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_qoy#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, sumsales#32 ASC NULLS FIRST, rk#33 ASC NULLS FIRST], [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] +(29) TakeOrderedAndProject +Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31, rk#32] +Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_qoy#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#32 ASC NULLS FIRST], [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31, rk#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (32) -+- * ColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan parquet spark_catalog.default.date_dim (28) +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) (unknown) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#33, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(29) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#7)) +(31) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#33, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1200)) AND (d_month_seq#33 <= 1211)) AND isnotnull(d_date_sk#7)) -(30) CometProject -Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] +(32) CometProject +Input [5]: [d_date_sk#7, d_month_seq#33, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(31) ColumnarToRow [codegen id : 1] +(33) ColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(32) BroadcastExchange +(34) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index 119ae71840..3fd090f51e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -1,16 +1,16 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (7) + WholeStageCodegen (6) Filter [rk] InputAdapter Window [sumsales,i_category] - WholeStageCodegen (6) - Sort [i_category,sumsales] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [i_category] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + CometSort [i_category,sumsales] + CometColumnarExchange [i_category] #1 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + RowToColumnar WholeStageCodegen (4) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index d87d02e87b..5b01bcc0da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -1,43 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * HashAggregate (29) - : : +- Exchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (33) - : +- * ColumnarToRow (32) - : +- CometFilter (31) - : +- CometScan parquet spark_catalog.default.customer (30) - +- ReusedExchange (36) +TakeOrderedAndProject (41) ++- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * ColumnarToRow (31) + : : +- CometHashAggregate (30) + : : +- CometColumnarExchange (29) + : : +- RowToColumnar (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * Project (13) + : : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : : :- * Project (6) + : : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- ReusedExchange (4) + : : : : +- BroadcastExchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store (7) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.household_demographics (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.customer_address (21) + : +- BroadcastExchange (35) + : +- * ColumnarToRow (34) + : +- CometFilter (33) + : +- CometScan parquet spark_catalog.default.customer (32) + +- ReusedExchange (38) (unknown) Scan parquet spark_catalog.default.store_sales @@ -55,7 +57,7 @@ Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotn (3) ColumnarToRow [codegen id : 5] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -(4) ReusedExchange [Reuses operator id: 44] +(4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#11] (5) BroadcastHashJoin [codegen id : 5] @@ -167,91 +169,95 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(Un Aggregate Attributes [3]: [sum#19, sum#20, sum#21] Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -(28) Exchange +(28) RowToColumnar Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) HashAggregate [codegen id : 8] +(29) CometColumnarExchange +Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometHashAggregate Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#25, sum(UnscaledValue(ss_ext_list_price#7))#26, sum(UnscaledValue(ss_ext_tax#8))#27] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#18 AS bought_city#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#25,17,2) AS extended_price#29, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#26,17,2) AS list_price#30, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#27,17,2) AS extended_tax#31] + +(31) ColumnarToRow [codegen id : 8] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#25, extended_price#26, list_price#27, extended_tax#28] (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +Output [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) +(33) CometFilter +Input [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#30)) -(32) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +(34) ColumnarToRow [codegen id : 6] +Input [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] -(33) BroadcastExchange -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +(35) BroadcastExchange +Input [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#32] +Right keys [1]: [c_customer_sk#29] Join type: Inner Join condition: None -(35) Project [codegen id : 8] -Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +(37) Project [codegen id : 8] +Output [8]: [ss_ticket_number#5, bought_city#25, extended_price#26, list_price#27, extended_tax#28, c_current_addr_sk#30, c_first_name#31, c_last_name#32] +Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#25, extended_price#26, list_price#27, extended_tax#28, c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] -(36) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#36, ca_city#37] +(38) ReusedExchange [Reuses operator id: 24] +Output [2]: [ca_address_sk#33, ca_city#34] -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#36] +(39) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [c_current_addr_sk#30] +Right keys [1]: [ca_address_sk#33] Join type: Inner -Join condition: NOT (ca_city#37 = bought_city#28) +Join condition: NOT (ca_city#34 = bought_city#25) -(38) Project [codegen id : 8] -Output [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35, ca_address_sk#36, ca_city#37] +(40) Project [codegen id : 8] +Output [8]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#25, ss_ticket_number#5, extended_price#26, extended_tax#28, list_price#27] +Input [10]: [ss_ticket_number#5, bought_city#25, extended_price#26, list_price#27, extended_tax#28, c_current_addr_sk#30, c_first_name#31, c_last_name#32, ca_address_sk#33, ca_city#34] -(39) TakeOrderedAndProject -Input [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Arguments: 100, [c_last_name#35 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] +(41) TakeOrderedAndProject +Input [8]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#25, ss_ticket_number#5, extended_price#26, extended_tax#28, list_price#27] +Arguments: 100, [c_last_name#32 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#32, c_first_name#31, ca_city#34, bought_city#25, ss_ticket_number#5, extended_price#26, extended_tax#28, list_price#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#38, d_dom#39] +Output [3]: [d_date_sk#11, d_year#35, d_dom#36] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [3]: [d_date_sk#11, d_year#38, d_dom#39] -Condition : ((((isnotnull(d_dom#39) AND (d_dom#39 >= 1)) AND (d_dom#39 <= 2)) AND d_year#38 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) +(43) CometFilter +Input [3]: [d_date_sk#11, d_year#35, d_dom#36] +Condition : ((((isnotnull(d_dom#36) AND (d_dom#36 >= 1)) AND (d_dom#36 <= 2)) AND d_year#35 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) -(42) CometProject -Input [3]: [d_date_sk#11, d_year#38, d_dom#39] +(44) CometProject +Input [3]: [d_date_sk#11, d_year#35, d_dom#36] Arguments: [d_date_sk#11], [d_date_sk#11] -(43) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(44) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt index f2680bebb0..a4396d582b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -4,56 +4,58 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] + ColumnarToRow InputAdapter - Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_city,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index 9a32627233..6c289ac77b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (25) - : : +- * BroadcastHashJoin LeftAnti BuildRight (24) - : : :- * BroadcastHashJoin LeftAnti BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.customer_address (26) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +* ColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (25) + : : +- * BroadcastHashJoin LeftAnti BuildRight (24) + : : :- * BroadcastHashJoin LeftAnti BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometProject (28) + : +- CometFilter (27) + : +- CometScan parquet spark_catalog.default.customer_address (26) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_demographics (33) (unknown) Scan parquet spark_catalog.default.customer @@ -67,7 +69,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) ReusedExchange [Reuses operator id: 47] +(6) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#7] (7) BroadcastHashJoin [codegen id : 2] @@ -100,7 +102,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -(13) ReusedExchange [Reuses operator id: 47] +(13) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#11] (14) BroadcastHashJoin [codegen id : 4] @@ -133,7 +135,7 @@ ReadSchema: struct (19) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] -(20) ReusedExchange [Reuses operator id: 47] +(20) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#15] (21) BroadcastHashJoin [codegen id : 6] @@ -227,50 +229,54 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#24] Results [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] -(40) Exchange +(40) RowToColumnar Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] -Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) HashAggregate [codegen id : 10] +(41) CometColumnarExchange +Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] +Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(42) CometHashAggregate Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] Keys [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#26] -Results [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, count(1)#26 AS cnt1#27, cd_purchase_estimate#22, count(1)#26 AS cnt2#28, cd_credit_rating#23, count(1)#26 AS cnt3#29] -(42) TakeOrderedAndProject -Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] -Arguments: 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] +(43) CometTakeOrderedAndProject +Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#26, cd_purchase_estimate#22, cnt2#27, cd_credit_rating#23, cnt3#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#19 ASC NULLS FIRST,cd_marital_status#20 ASC NULLS FIRST,cd_education_status#21 ASC NULLS FIRST,cd_purchase_estimate#22 ASC NULLS FIRST,cd_credit_rating#23 ASC NULLS FIRST], output=[cd_gender#19,cd_marital_status#20,cd_education_status#21,cnt1#26,cd_purchase_estimate#22,cnt2#27,cd_credit_rating#23,cnt3#28]), 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#26, cd_purchase_estimate#22, cnt2#27, cd_credit_rating#23, cnt3#28] + +(44) ColumnarToRow [codegen id : 10] +Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#26, cd_purchase_estimate#22, cnt2#27, cd_credit_rating#23, cnt3#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#30, d_moy#31] +Output [3]: [d_date_sk#7, d_year#29, d_moy#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter -Input [3]: [d_date_sk#7, d_year#30, d_moy#31] -Condition : (((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2001)) AND (d_moy#31 >= 4)) AND (d_moy#31 <= 6)) AND isnotnull(d_date_sk#7)) +(46) CometFilter +Input [3]: [d_date_sk#7, d_year#29, d_moy#30] +Condition : (((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 >= 4)) AND (d_moy#30 <= 6)) AND isnotnull(d_date_sk#7)) -(45) CometProject -Input [3]: [d_date_sk#7, d_year#30, d_moy#31] +(47) CometProject +Input [3]: [d_date_sk#7, d_year#29, d_moy#30] Arguments: [d_date_sk#7], [d_date_sk#7] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index f5b4eccfbe..410950aeb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -1,74 +1,76 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - InputAdapter - Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow +WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] + CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt index 18ff7c4590..31a4d6acdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt @@ -1,34 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan parquet spark_catalog.default.promotion (20) +* ColumnarToRow (32) ++- CometTakeOrderedAndProject (31) + +- CometHashAggregate (30) + +- CometColumnarExchange (29) + +- RowToColumnar (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * ColumnarToRow (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : +- ReusedExchange (11) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.item (14) + +- BroadcastExchange (24) + +- * ColumnarToRow (23) + +- CometProject (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.promotion (20) (unknown) Scan parquet spark_catalog.default.store_sales @@ -78,7 +80,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(11) ReusedExchange [Reuses operator id: 35] +(11) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#14] (12) BroadcastHashJoin [codegen id : 5] @@ -158,50 +160,54 @@ Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_pr Aggregate Attributes [8]: [sum#20, count#21, sum#22, count#23, sum#24, count#25, sum#26, count#27] Results [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -(28) Exchange +(28) RowToColumnar Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) HashAggregate [codegen id : 6] +(29) CometColumnarExchange +Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometHashAggregate Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] Keys [1]: [i_item_id#16] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#36, avg(UnscaledValue(ss_list_price#5))#37, avg(UnscaledValue(ss_coupon_amt#7))#38, avg(UnscaledValue(ss_sales_price#6))#39] -Results [5]: [i_item_id#16, avg(ss_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(ss_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(ss_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(ss_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] -(30) TakeOrderedAndProject -Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] -Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] +(31) CometTakeOrderedAndProject +Input [5]: [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#16 ASC NULLS FIRST], output=[i_item_id#16,agg1#36,agg2#37,agg3#38,agg4#39]), 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] + +(32) ColumnarToRow [codegen id : 6] +Input [5]: [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#44] +Output [2]: [d_date_sk#14, d_year#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter -Input [2]: [d_date_sk#14, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) +(34) CometFilter +Input [2]: [d_date_sk#14, d_year#40] +Condition : ((isnotnull(d_year#40) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#14)) -(33) CometProject -Input [2]: [d_date_sk#14, d_year#44] +(35) CometProject +Input [2]: [d_date_sk#14, d_year#40] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(35) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt index 2471de20a3..37d5a50ae0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt @@ -1,52 +1,54 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange [i_item_id] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometProject [p_promo_sk] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 32499fad8a..8387487b4f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -1,46 +1,50 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- Window (40) - +- * Sort (39) - +- Exchange (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Expand (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (31) - +- * BroadcastHashJoin LeftSemi BuildRight (30) - :- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (29) - +- * Project (28) - +- * Filter (27) - +- Window (26) - +- * Sort (25) - +- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.store (13) - +- ReusedExchange (19) +TakeOrderedAndProject (46) ++- * Project (45) + +- Window (44) + +- * ColumnarToRow (43) + +- CometSort (42) + +- CometColumnarExchange (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- RowToColumnar (38) + +- * HashAggregate (37) + +- * Expand (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- ReusedExchange (4) + +- BroadcastExchange (33) + +- * BroadcastHashJoin LeftSemi BuildRight (32) + :- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.store (7) + +- BroadcastExchange (31) + +- * Project (30) + +- * Filter (29) + +- Window (28) + +- * ColumnarToRow (27) + +- CometSort (26) + +- CometHashAggregate (25) + +- CometColumnarExchange (24) + +- RowToColumnar (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildRight (20) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store_sales (10) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.store (13) + +- ReusedExchange (19) (unknown) Scan parquet spark_catalog.default.store_sales @@ -58,7 +62,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 8] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 47] +(4) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 8] @@ -128,7 +132,7 @@ Join condition: None Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] -(19) ReusedExchange [Reuses operator id: 47] +(19) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#15] (20) BroadcastHashJoin [codegen id : 4] @@ -148,128 +152,136 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum#16] Results [2]: [s_state#14, sum#17] -(23) Exchange +(23) RowToColumnar Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(24) HashAggregate [codegen id : 5] +(24) CometColumnarExchange +Input [2]: [s_state#14, sum#17] +Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(25) CometHashAggregate Input [2]: [s_state#14, sum#17] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] -Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] -(25) Sort [codegen id : 5] -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 +(26) CometSort +Input [3]: [s_state#14, _w0#18, s_state#14] +Arguments: [s_state#14, _w0#18, s_state#14], [s_state#14 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(26) Window -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] +(27) ColumnarToRow [codegen id : 5] +Input [3]: [s_state#14, _w0#18, s_state#14] -(27) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] -Condition : (ranking#20 <= 5) +(28) Window +Input [3]: [s_state#14, _w0#18, s_state#14] +Arguments: [rank(_w0#18) windowspecdefinition(s_state#14, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#14], [_w0#18 DESC NULLS LAST] -(28) Project [codegen id : 6] +(29) Filter [codegen id : 6] +Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] +Condition : (ranking#19 <= 5) + +(30) Project [codegen id : 6] Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] +Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] -(29) BroadcastExchange +(31) BroadcastExchange Input [1]: [s_state#14] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 7] Left keys [1]: [s_state#8] Right keys [1]: [s_state#14] Join type: LeftSemi Join condition: None -(31) BroadcastExchange +(33) BroadcastExchange Input [3]: [s_store_sk#6, s_county#7, s_state#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(35) Project [codegen id : 8] Output [3]: [ss_net_profit#2, s_state#8, s_county#7] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] -(34) Expand [codegen id : 8] +(36) Expand [codegen id : 8] Input [3]: [ss_net_profit#2, s_state#8, s_county#7] -Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] +Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22] -(35) HashAggregate [codegen id : 8] -Input [4]: [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] -Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] +(37) HashAggregate [codegen id : 8] +Input [4]: [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22] +Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#24] -Results [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] +Aggregate Attributes [1]: [sum#23] +Results [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] + +(38) RowToColumnar +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] -(36) Exchange -Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] -Arguments: hashpartitioning(s_state#21, s_county#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(39) CometColumnarExchange +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] +Arguments: hashpartitioning(s_state#20, s_county#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) HashAggregate [codegen id : 9] -Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] -Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] +(40) CometHashAggregate +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] +Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#26] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS total_sum#27, s_state#21, s_county#22, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS lochierarchy#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS _w0#29, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS _w1#30, CASE WHEN (cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint) = 0) THEN s_state#21 END AS _w2#31] -(38) Exchange -Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] -Arguments: hashpartitioning(_w1#30, _w2#31, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(41) CometColumnarExchange +Input [7]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29] +Arguments: hashpartitioning(_w1#28, _w2#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(42) CometSort +Input [7]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29] +Arguments: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29], [_w1#28 ASC NULLS FIRST, _w2#29 ASC NULLS FIRST, _w0#27 DESC NULLS LAST] -(39) Sort [codegen id : 10] -Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] -Arguments: [_w1#30 ASC NULLS FIRST, _w2#31 ASC NULLS FIRST, _w0#29 DESC NULLS LAST], false, 0 +(43) ColumnarToRow [codegen id : 9] +Input [7]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29] -(40) Window -Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] -Arguments: [rank(_w0#29) windowspecdefinition(_w1#30, _w2#31, _w0#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#32], [_w1#30, _w2#31], [_w0#29 DESC NULLS LAST] +(44) Window +Input [7]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29] +Arguments: [rank(_w0#27) windowspecdefinition(_w1#28, _w2#29, _w0#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#30], [_w1#28, _w2#29], [_w0#27 DESC NULLS LAST] -(41) Project [codegen id : 11] -Output [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] -Input [8]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31, rank_within_parent#32] +(45) Project [codegen id : 10] +Output [5]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, rank_within_parent#30] +Input [8]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29, rank_within_parent#30] -(42) TakeOrderedAndProject -Input [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] -Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN s_state#21 END ASC NULLS FIRST, rank_within_parent#32 ASC NULLS FIRST], [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] +(46) TakeOrderedAndProject +Input [5]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, rank_within_parent#30] +Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#20 END ASC NULLS FIRST, rank_within_parent#30 ASC NULLS FIRST], [total_sum#25, s_state#20, s_county#21, lochierarchy#26, rank_within_parent#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#33] +Output [2]: [d_date_sk#5, d_month_seq#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#33] -Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1200)) AND (d_month_seq#33 <= 1211)) AND isnotnull(d_date_sk#5)) +(48) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#31] +Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1200)) AND (d_month_seq#31 <= 1211)) AND isnotnull(d_date_sk#5)) -(45) CometProject -Input [2]: [d_date_sk#5, d_month_seq#33] +(49) CometProject +Input [2]: [d_date_sk#5, d_month_seq#31] Arguments: [d_date_sk#5], [d_date_sk#5] -(46) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(47) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index 0e01c5f710..53a93dc43f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -1,16 +1,16 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (11) + WholeStageCodegen (10) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (10) - Sort [_w1,_w2,_w0] + WholeStageCodegen (9) + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - WholeStageCodegen (9) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - InputAdapter - Exchange [s_state,s_county,spark_grouping_id] #2 + CometSort [_w1,_w2,_w0] + CometColumnarExchange [_w1,_w2] #1 + CometHashAggregate [s_state,s_county,spark_grouping_id,sum] + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + RowToColumnar WholeStageCodegen (8) HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] Expand [ss_net_profit,s_state,s_county] @@ -48,27 +48,29 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter Window [_w0,s_state] WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #6 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [s_state,sum] + CometColumnarExchange [s_state] #6 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 39bedd1f2c..172f460c8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -1,42 +1,44 @@ == Physical Plan == -* Sort (38) -+- Exchange (37) - +- * HashAggregate (36) - +- Exchange (35) - +- * HashAggregate (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildLeft (25) - : :- BroadcastExchange (5) - : : +- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.item (1) - : +- Union (24) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.web_sales (6) - : : +- ReusedExchange (9) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * ColumnarToRow (14) - : : : +- CometFilter (13) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : +- ReusedExchange (15) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * ColumnarToRow (20) - : : +- CometFilter (19) - : : +- CometScan parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (21) - +- BroadcastExchange (31) - +- * ColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.time_dim (27) +* ColumnarToRow (40) ++- CometSort (39) + +- CometColumnarExchange (38) + +- CometHashAggregate (37) + +- CometColumnarExchange (36) + +- RowToColumnar (35) + +- * HashAggregate (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildLeft (25) + : :- BroadcastExchange (5) + : : +- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.item (1) + : +- Union (24) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.web_sales (6) + : : +- ReusedExchange (9) + : :- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * ColumnarToRow (14) + : : : +- CometFilter (13) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : +- ReusedExchange (15) + : +- * Project (23) + : +- * BroadcastHashJoin Inner BuildRight (22) + : :- * ColumnarToRow (20) + : : +- CometFilter (19) + : : +- CometScan parquet spark_catalog.default.store_sales (18) + : +- ReusedExchange (21) + +- BroadcastExchange (31) + +- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.time_dim (27) (unknown) Scan parquet spark_catalog.default.item @@ -76,7 +78,7 @@ Condition : (isnotnull(ws_item_sk#6) AND isnotnull(ws_sold_time_sk#5)) (8) ColumnarToRow [codegen id : 3] Input [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] -(9) ReusedExchange [Reuses operator id: 43] +(9) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#10] (10) BroadcastHashJoin [codegen id : 3] @@ -104,7 +106,7 @@ Condition : (isnotnull(cs_item_sk#15) AND isnotnull(cs_sold_time_sk#14)) (14) ColumnarToRow [codegen id : 5] Input [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] -(15) ReusedExchange [Reuses operator id: 43] +(15) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#19] (16) BroadcastHashJoin [codegen id : 5] @@ -132,7 +134,7 @@ Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) (20) ColumnarToRow [codegen id : 7] Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(21) ReusedExchange [Reuses operator id: 43] +(21) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#28] (22) BroadcastHashJoin [codegen id : 7] @@ -196,54 +198,58 @@ Functions [1]: [partial_sum(UnscaledValue(ext_price#11))] Aggregate Attributes [1]: [sum#36] Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] -(35) Exchange +(35) RowToColumnar Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(36) HashAggregate [codegen id : 10] +(36) CometColumnarExchange +Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(37) CometHashAggregate Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] Functions [1]: [sum(UnscaledValue(ext_price#11))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#11))#38] -Results [5]: [i_brand_id#2 AS brand_id#39, i_brand#3 AS brand#40, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#11))#38,17,2) AS ext_price#41] -(37) Exchange -Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] -Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(38) CometColumnarExchange +Input [5]: [brand_id#38, brand#39, t_hour#33, t_minute#34, ext_price#40] +Arguments: rangepartitioning(ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) CometSort +Input [5]: [brand_id#38, brand#39, t_hour#33, t_minute#34, ext_price#40] +Arguments: [brand_id#38, brand#39, t_hour#33, t_minute#34, ext_price#40], [ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST] -(38) Sort [codegen id : 11] -Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] -Arguments: [ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST], true, 0 +(40) ColumnarToRow [codegen id : 10] +Input [5]: [brand_id#38, brand#39, t_hour#33, t_minute#34, ext_price#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (43) -+- * ColumnarToRow (42) - +- CometProject (41) - +- CometFilter (40) - +- CometScan parquet spark_catalog.default.date_dim (39) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#42, d_moy#43] +Output [3]: [d_date_sk#10, d_year#41, d_moy#42] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(40) CometFilter -Input [3]: [d_date_sk#10, d_year#42, d_moy#43] -Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 11)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#10)) +(42) CometFilter +Input [3]: [d_date_sk#10, d_year#41, d_moy#42] +Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 11)) AND (d_year#41 = 1999)) AND isnotnull(d_date_sk#10)) -(41) CometProject -Input [3]: [d_date_sk#10, d_year#42, d_moy#43] +(43) CometProject +Input [3]: [d_date_sk#10, d_year#41, d_moy#42] Arguments: [d_date_sk#10], [d_date_sk#10] -(42) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(43) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index bea5376a00..b5c6e96d6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -1,11 +1,11 @@ -WholeStageCodegen (11) - Sort [ext_price,brand_id] +WholeStageCodegen (10) + ColumnarToRow InputAdapter - Exchange [ext_price,brand_id] #1 - WholeStageCodegen (10) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] - InputAdapter - Exchange [i_brand,i_brand_id,t_hour,t_minute] #2 + CometSort [ext_price,brand_id] + CometColumnarExchange [ext_price,brand_id] #1 + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] + CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 + RowToColumnar WholeStageCodegen (9) HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index c0d1f949a8..dbc785238b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -1,74 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * SortMergeJoin LeftOuter (65) - :- * Sort (58) - : +- Exchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * ColumnarToRow (3) - : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * ColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * ColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * ColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * ColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * ColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * ColumnarToRow (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.promotion (51) - +- * ColumnarToRow (64) - +- CometSort (63) - +- CometExchange (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.catalog_returns (59) +* ColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometHashAggregate (69) + +- CometColumnarExchange (68) + +- CometHashAggregate (67) + +- CometProject (66) + +- CometSortMergeJoin (65) + :- CometSort (59) + : +- CometColumnarExchange (58) + : +- RowToColumnar (57) + : +- * Project (56) + : +- * BroadcastHashJoin LeftOuter BuildRight (55) + : :- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Project (15) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : : :- * Project (9) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : : : : :- * ColumnarToRow (3) + : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- BroadcastExchange (7) + : : : : : : : : : +- * ColumnarToRow (6) + : : : : : : : : : +- CometFilter (5) + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) + : : : : : : : : +- BroadcastExchange (13) + : : : : : : : : +- * ColumnarToRow (12) + : : : : : : : : +- CometFilter (11) + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * ColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- CometProject (24) + : : : : : : +- CometFilter (23) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) + : : : : : +- BroadcastExchange (33) + : : : : : +- * ColumnarToRow (32) + : : : : : +- CometProject (31) + : : : : : +- CometFilter (30) + : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) + : : : : +- ReusedExchange (36) + : : : +- BroadcastExchange (42) + : : : +- * ColumnarToRow (41) + : : : +- CometFilter (40) + : : : +- CometScan parquet spark_catalog.default.date_dim (39) + : : +- BroadcastExchange (48) + : : +- * ColumnarToRow (47) + : : +- CometFilter (46) + : : +- CometScan parquet spark_catalog.default.date_dim (45) + : +- BroadcastExchange (54) + : +- * ColumnarToRow (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.promotion (51) + +- CometSort (64) + +- CometColumnarExchange (63) + +- CometProject (62) + +- CometFilter (61) + +- CometScan parquet spark_catalog.default.catalog_returns (60) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -235,7 +236,7 @@ Join condition: None Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -(36) ReusedExchange [Reuses operator id: 75] +(36) ReusedExchange [Reuses operator id: 76] Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (37) BroadcastHashJoin [codegen id : 10] @@ -332,13 +333,16 @@ Join condition: None Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] -(57) Exchange +(57) RowToColumnar Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(58) Sort [codegen id : 11] +(58) CometColumnarExchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(59) CometSort +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.catalog_returns Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] @@ -347,86 +351,81 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) CometFilter +(61) CometFilter Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) -(61) CometProject +(62) CometProject Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] -(62) CometExchange +(63) CometColumnarExchange Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(63) CometSort +(64) CometSort Input [2]: [cr_item_sk#30, cr_order_number#31] Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] -(64) ColumnarToRow [codegen id : 12] -Input [2]: [cr_item_sk#30, cr_order_number#31] +(65) CometSortMergeJoin +Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Right output [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter -(65) SortMergeJoin [codegen id : 13] -Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#30, cr_order_number#31] -Join type: LeftOuter -Join condition: None - -(66) Project [codegen id : 13] -Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(66) CometProject Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] +Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(67) HashAggregate [codegen id : 13] +(67) CometHashAggregate Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -(68) Exchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(68) CometColumnarExchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(69) HashAggregate [codegen id : 14] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +(69) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#35 AS no_promo#36, count(1)#35 AS promo#37, count(1)#35 AS total_cnt#38] -(70) TakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] -Arguments: 100, [total_cnt#38 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] +(70) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), 100, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] + +(71) ColumnarToRow [codegen id : 11] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] -Condition : ((((isnotnull(d_year#39) AND (d_year#39 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(73) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +(74) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(75) BroadcastExchange +(76) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index 5eb8ea5275..5bf7ab343a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -1,114 +1,108 @@ -TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - WholeStageCodegen (14) - HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] - InputAdapter - Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - WholeStageCodegen (13) - HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] - Project [w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (11) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - ColumnarToRow - InputAdapter - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] +WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] + CometColumnarExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq] + CometProject [w_warehouse_name,i_item_desc,d_week_seq] + CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + RowToColumnar + WholeStageCodegen (10) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + ColumnarToRow + InputAdapter + CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) + BroadcastExchange #6 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometProject [cd_demo_sk] + CometFilter [cd_marital_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + BroadcastExchange #8 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [d_week_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) + BroadcastExchange #9 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometFilter [d_week_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) + BroadcastExchange #10 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index dc64c33861..38bf356802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt @@ -1,36 +1,40 @@ == Physical Plan == -* Sort (32) -+- Exchange (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (24) - : +- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (28) - +- * ColumnarToRow (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.customer (25) +* ColumnarToRow (36) ++- CometSort (35) + +- CometColumnarExchange (34) + +- RowToColumnar (33) + +- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * ColumnarToRow (26) + : +- CometFilter (25) + : +- CometHashAggregate (24) + : +- CometColumnarExchange (23) + : +- RowToColumnar (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (30) + +- * ColumnarToRow (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.customer (27) (unknown) Scan parquet spark_catalog.default.store_sales @@ -48,7 +52,7 @@ Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnu (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 37] +(4) ReusedExchange [Reuses operator id: 41] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -132,86 +136,96 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#14] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) Exchange +(22) RowToColumnar Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(23) HashAggregate [codegen id : 6] +(23) CometColumnarExchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(24) CometHashAggregate Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(24) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) +(25) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] +Condition : ((cnt#16 >= 1) AND (cnt#16 <= 5)) + +(26) ColumnarToRow [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] (unknown) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Output [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) +(28) CometFilter +Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Condition : isnotnull(c_customer_sk#17) -(27) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(29) ColumnarToRow [codegen id : 5] +Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] -(28) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(30) BroadcastExchange +Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#17] Join type: Inner Join condition: None -(30) Project [codegen id : 6] -Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(32) Project [codegen id : 6] +Output [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16, c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] + +(33) RowToColumnar +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] + +(34) CometColumnarExchange +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Arguments: rangepartitioning(cnt#16 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(31) Exchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(35) CometSort +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Arguments: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16], [cnt#16 DESC NULLS LAST] -(32) Sort [codegen id : 7] -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [cnt#17 DESC NULLS LAST], true, 0 +(36) ColumnarToRow [codegen id : 7] +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (41) ++- * ColumnarToRow (40) + +- CometProject (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.date_dim (37) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +Output [3]: [d_date_sk#7, d_year#22, d_dom#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] -Condition : ((((isnotnull(d_dom#24) AND (d_dom#24 >= 1)) AND (d_dom#24 <= 2)) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(38) CometFilter +Input [3]: [d_date_sk#7, d_year#22, d_dom#23] +Condition : ((((isnotnull(d_dom#23) AND (d_dom#23 >= 1)) AND (d_dom#23 <= 2)) AND d_year#22 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(35) CometProject -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +(39) CometProject +Input [3]: [d_date_sk#7, d_year#22, d_dom#23] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(40) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(37) BroadcastExchange +(41) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index 7c5ee1ef5a..45a7796f70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -1,56 +1,60 @@ WholeStageCodegen (7) - Sort [cnt] + ColumnarToRow InputAdapter - Exchange [cnt] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - InputAdapter - Exchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow + CometSort [cnt] + CometColumnarExchange [cnt] #1 + RowToColumnar + WholeStageCodegen (6) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index 85413ac1ce..431ca36ddf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -1,75 +1,83 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * ColumnarToRow (36) - : : : +- CometFilter (35) - : : : +- CometScan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) - +- BroadcastExchange (68) - +- * HashAggregate (67) - +- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * ColumnarToRow (55) - : : +- CometFilter (54) - : : +- CometScan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) +TakeOrderedAndProject (79) ++- * Project (78) + +- * BroadcastHashJoin Inner BuildRight (77) + :- * Project (58) + : +- * BroadcastHashJoin Inner BuildRight (57) + : :- * BroadcastHashJoin Inner BuildRight (37) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometColumnarExchange (15) + : : : +- RowToColumnar (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (36) + : : +- * ColumnarToRow (35) + : : +- CometHashAggregate (34) + : : +- CometColumnarExchange (33) + : : +- RowToColumnar (32) + : : +- * HashAggregate (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * Project (27) + : : : +- * BroadcastHashJoin Inner BuildRight (26) + : : : :- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.store_sales (22) + : : +- ReusedExchange (28) + : +- BroadcastExchange (56) + : +- * ColumnarToRow (55) + : +- CometFilter (54) + : +- CometHashAggregate (53) + : +- CometColumnarExchange (52) + : +- RowToColumnar (51) + : +- * HashAggregate (50) + : +- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (46) + : : +- * BroadcastHashJoin Inner BuildRight (45) + : : :- * ColumnarToRow (40) + : : : +- CometFilter (39) + : : : +- CometScan parquet spark_catalog.default.customer (38) + : : +- BroadcastExchange (44) + : : +- * ColumnarToRow (43) + : : +- CometFilter (42) + : : +- CometScan parquet spark_catalog.default.web_sales (41) + : +- ReusedExchange (47) + +- BroadcastExchange (76) + +- * ColumnarToRow (75) + +- CometHashAggregate (74) + +- CometColumnarExchange (73) + +- RowToColumnar (72) + +- * HashAggregate (71) + +- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * ColumnarToRow (61) + : : +- CometFilter (60) + : : +- CometScan parquet spark_catalog.default.customer (59) + : +- BroadcastExchange (65) + : +- * ColumnarToRow (64) + : +- CometFilter (63) + : +- CometScan parquet spark_catalog.default.web_sales (62) + +- ReusedExchange (68) (unknown) Scan parquet spark_catalog.default.customer @@ -115,7 +123,7 @@ Join condition: None Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 75] +(10) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#9, d_year#10] (11) BroadcastHashJoin [codegen id : 3] @@ -135,296 +143,312 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] Aggregate Attributes [1]: [sum#11] Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -(14) Exchange +(14) RowToColumnar Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 16] +(15) CometColumnarExchange +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] -Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] -(16) Filter [codegen id : 16] -Input [2]: [customer_id#14, year_total#15] -Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) +(17) CometFilter +Input [2]: [customer_id#13, year_total#14] +Condition : (isnotnull(year_total#14) AND (year_total#14 > 0.00)) + +(18) ColumnarToRow [codegen id : 16] +Input [2]: [customer_id#13, year_total#14] (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +Output [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter -Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] -Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) +(20) CometFilter +Input [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_customer_id#16)) -(19) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +(21) ColumnarToRow [codegen id : 6] +Input [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#20) +(23) CometFilter +Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Condition : isnotnull(ss_customer_sk#19) -(22) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +(24) ColumnarToRow [codegen id : 4] +Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -(23) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +(25) BroadcastExchange +Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#16] -Right keys [1]: [ss_customer_sk#20] +(26) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#15] +Right keys [1]: [ss_customer_sk#19] Join type: Inner Join condition: None -(25) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] -Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +(27) Project [codegen id : 6] +Output [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21] +Input [7]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18, ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -(26) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#24, d_year#25] +(28) ReusedExchange [Reuses operator id: 87] +Output [2]: [d_date_sk#23, d_year#24] -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#24] +(29) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#21] +Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] - -(29) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] -Aggregate Attributes [1]: [sum#26] -Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] - -(30) Exchange -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] -Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] -Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] - -(32) BroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +(30) Project [codegen id : 6] +Output [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] +Input [7]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21, d_date_sk#23, d_year#24] + +(31) HashAggregate [codegen id : 6] +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] +Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#20))] +Aggregate Attributes [1]: [sum#25] +Results [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] + +(32) RowToColumnar +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] + +(33) CometColumnarExchange +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] +Arguments: hashpartitioning(c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometHashAggregate +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] +Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] +Functions [1]: [sum(UnscaledValue(ss_net_paid#20))] + +(35) ColumnarToRow [codegen id : 7] +Input [4]: [customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30] + +(36) BroadcastExchange +Input [4]: [customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#28] +(37) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#27] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Output [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) +(39) CometFilter +Input [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] +Condition : (isnotnull(c_customer_sk#31) AND isnotnull(c_customer_id#32)) -(36) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +(40) ColumnarToRow [codegen id : 10] +Input [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Output [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_bill_customer_sk#36) +(42) CometFilter +Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] +Condition : isnotnull(ws_bill_customer_sk#35) -(39) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(43) ColumnarToRow [codegen id : 8] +Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] -(40) BroadcastExchange -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(44) BroadcastExchange +Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#32] -Right keys [1]: [ws_bill_customer_sk#36] +(45) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#31] +Right keys [1]: [ws_bill_customer_sk#35] Join type: Inner Join condition: None -(42) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] -Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(46) Project [codegen id : 10] +Output [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, ws_sold_date_sk#37] +Input [7]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] -(43) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#40, d_year#41] +(47) ReusedExchange [Reuses operator id: 83] +Output [2]: [d_date_sk#39, d_year#40] -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#37] +Right keys [1]: [d_date_sk#39] Join type: Inner Join condition: None -(45) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] - -(46) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] -Aggregate Attributes [1]: [sum#42] -Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] - -(47) Exchange -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(48) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] -Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] -Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] - -(49) Filter [codegen id : 11] -Input [2]: [customer_id#45, year_total#46] -Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) - -(50) BroadcastExchange -Input [2]: [customer_id#45, year_total#46] +(49) Project [codegen id : 10] +Output [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, d_year#40] +Input [7]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, ws_sold_date_sk#37, d_date_sk#39, d_year#40] + +(50) HashAggregate [codegen id : 10] +Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, d_year#40] +Keys [4]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#36))] +Aggregate Attributes [1]: [sum#41] +Results [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] + +(51) RowToColumnar +Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] + +(52) CometColumnarExchange +Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] +Arguments: hashpartitioning(c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(53) CometHashAggregate +Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] +Keys [4]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40] +Functions [1]: [sum(UnscaledValue(ws_net_paid#36))] + +(54) CometFilter +Input [2]: [customer_id#43, year_total#44] +Condition : (isnotnull(year_total#44) AND (year_total#44 > 0.00)) + +(55) ColumnarToRow [codegen id : 11] +Input [2]: [customer_id#43, year_total#44] + +(56) BroadcastExchange +Input [2]: [customer_id#43, year_total#44] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#45] +(57) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#43] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] -Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] +(58) Project [codegen id : 16] +Output [7]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, year_total#44] +Input [8]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, customer_id#43, year_total#44] (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +Output [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter -Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] -Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) +(60) CometFilter +Input [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] +Condition : (isnotnull(c_customer_sk#45) AND isnotnull(c_customer_id#46)) -(55) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +(61) ColumnarToRow [codegen id : 14] +Input [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Output [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#52)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_bill_customer_sk#51) +(63) CometFilter +Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_customer_sk#49) -(58) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +(64) ColumnarToRow [codegen id : 12] +Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] -(59) BroadcastExchange -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +(65) BroadcastExchange +Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#47] -Right keys [1]: [ws_bill_customer_sk#51] +(66) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#45] +Right keys [1]: [ws_bill_customer_sk#49] Join type: Inner Join condition: None -(61) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] -Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +(67) Project [codegen id : 14] +Output [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, ws_sold_date_sk#51] +Input [7]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48, ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] -(62) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#55, d_year#56] +(68) ReusedExchange [Reuses operator id: 87] +Output [2]: [d_date_sk#53, d_year#54] -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#53] -Right keys [1]: [d_date_sk#55] +(69) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#53] Join type: Inner Join condition: None -(64) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] - -(65) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] -Aggregate Attributes [1]: [sum#57] -Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] - -(66) Exchange -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(67) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] -Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#52))#44] -Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#52))#44,17,2) AS year_total#60] - -(68) BroadcastExchange -Input [2]: [customer_id#59, year_total#60] +(70) Project [codegen id : 14] +Output [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, d_year#54] +Input [7]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, ws_sold_date_sk#51, d_date_sk#53, d_year#54] + +(71) HashAggregate [codegen id : 14] +Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, d_year#54] +Keys [4]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#50))] +Aggregate Attributes [1]: [sum#55] +Results [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] + +(72) RowToColumnar +Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] + +(73) CometColumnarExchange +Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] +Arguments: hashpartitioning(c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(74) CometHashAggregate +Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] +Keys [4]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54] +Functions [1]: [sum(UnscaledValue(ws_net_paid#50))] + +(75) ColumnarToRow [codegen id : 15] +Input [2]: [customer_id#57, year_total#58] + +(76) BroadcastExchange +Input [2]: [customer_id#57, year_total#58] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(69) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#59] +(77) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#57] Join type: Inner -Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) +Join condition: (CASE WHEN (year_total#44 > 0.00) THEN (year_total#58 / year_total#44) END > CASE WHEN (year_total#14 > 0.00) THEN (year_total#30 / year_total#14) END) -(70) Project [codegen id : 16] -Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] +(78) Project [codegen id : 16] +Output [3]: [customer_id#27, customer_first_name#28, customer_last_name#29] +Input [9]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, year_total#44, customer_id#57, year_total#58] -(71) TakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +(79) TakeOrderedAndProject +Input [3]: [customer_id#27, customer_first_name#28, customer_last_name#29] +Arguments: 100, [customer_id#27 ASC NULLS FIRST, customer_id#27 ASC NULLS FIRST, customer_id#27 ASC NULLS FIRST], [customer_id#27, customer_first_name#28, customer_last_name#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (83) ++- * ColumnarToRow (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) (unknown) Scan parquet spark_catalog.default.date_dim @@ -434,44 +458,44 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(81) CometFilter Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(74) ColumnarToRow [codegen id : 1] +(82) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(75) BroadcastExchange +(83) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 +BroadcastExchange (87) ++- * ColumnarToRow (86) + +- CometFilter (85) + +- CometScan parquet spark_catalog.default.date_dim (84) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +Output [2]: [d_date_sk#23, d_year#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) +(85) CometFilter +Input [2]: [d_date_sk#23, d_year#24] +Condition : (((isnotnull(d_year#24) AND (d_year#24 = 2002)) AND d_year#24 IN (2001,2002)) AND isnotnull(d_date_sk#23)) -(78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#25] +(86) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#23, d_year#24] -(79) BroadcastExchange -Input [2]: [d_date_sk#24, d_year#25] +(87) BroadcastExchange +Input [2]: [d_date_sk#23, d_year#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#22 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 9d3ae8fbee..6136429173 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -5,43 +5,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [year_total] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + RowToColumnar + WholeStageCodegen (3) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -52,71 +22,109 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + ColumnarToRow + InputAdapter + CometFilter [year_total] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + ColumnarToRow InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index 3a29db88ef..8b2d451b09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -1,136 +1,138 @@ == Physical Plan == -TakeOrderedAndProject (132) -+- * Project (131) - +- * SortMergeJoin Inner (130) - :- * Sort (71) - : +- Exchange (70) - : +- * Filter (69) - : +- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- Union (62) - : :- * Project (23) - : : +- * SortMergeJoin LeftOuter (22) - : : :- * Sort (15) - : : : +- Exchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- * ColumnarToRow (21) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : :- * Project (42) - : : +- * SortMergeJoin LeftOuter (41) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : :- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * ColumnarToRow (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) - : : : : +- ReusedExchange (27) - : : : +- ReusedExchange (30) - : : +- * ColumnarToRow (40) - : : +- CometSort (39) - : : +- CometExchange (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometScan parquet spark_catalog.default.store_returns (35) - : +- * Project (61) - : +- * SortMergeJoin LeftOuter (60) - : :- * Sort (53) - : : +- Exchange (52) - : : +- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * ColumnarToRow (45) - : : : : +- CometFilter (44) - : : : : +- CometScan parquet spark_catalog.default.web_sales (43) - : : : +- ReusedExchange (46) - : : +- ReusedExchange (49) - : +- * ColumnarToRow (59) - : +- CometSort (58) - : +- CometExchange (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan parquet spark_catalog.default.web_returns (54) - +- * Sort (129) - +- Exchange (128) - +- * Filter (127) - +- * HashAggregate (126) - +- Exchange (125) - +- * HashAggregate (124) - +- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- Union (120) - :- * Project (87) - : +- * SortMergeJoin LeftOuter (86) - : :- * Sort (82) - : : +- Exchange (81) - : : +- * Project (80) - : : +- * BroadcastHashJoin Inner BuildRight (79) - : : :- * Project (77) - : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : :- * ColumnarToRow (74) - : : : : +- CometFilter (73) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) - : : : +- ReusedExchange (75) - : : +- ReusedExchange (78) - : +- * ColumnarToRow (85) - : +- CometSort (84) - : +- ReusedExchange (83) - :- * Project (103) - : +- * SortMergeJoin LeftOuter (102) - : :- * Sort (98) - : : +- Exchange (97) - : : +- * Project (96) - : : +- * BroadcastHashJoin Inner BuildRight (95) - : : :- * Project (93) - : : : +- * BroadcastHashJoin Inner BuildRight (92) - : : : :- * ColumnarToRow (90) - : : : : +- CometFilter (89) - : : : : +- CometScan parquet spark_catalog.default.store_sales (88) - : : : +- ReusedExchange (91) - : : +- ReusedExchange (94) - : +- * ColumnarToRow (101) - : +- CometSort (100) - : +- ReusedExchange (99) - +- * Project (119) - +- * SortMergeJoin LeftOuter (118) - :- * Sort (114) - : +- Exchange (113) - : +- * Project (112) - : +- * BroadcastHashJoin Inner BuildRight (111) - : :- * Project (109) - : : +- * BroadcastHashJoin Inner BuildRight (108) - : : :- * ColumnarToRow (106) - : : : +- CometFilter (105) - : : : +- CometScan parquet spark_catalog.default.web_sales (104) - : : +- ReusedExchange (107) - : +- ReusedExchange (110) - +- * ColumnarToRow (117) - +- CometSort (116) - +- ReusedExchange (115) +TakeOrderedAndProject (134) ++- * Project (133) + +- * SortMergeJoin Inner (132) + :- * ColumnarToRow (72) + : +- CometSort (71) + : +- CometColumnarExchange (70) + : +- CometFilter (69) + : +- CometHashAggregate (68) + : +- CometColumnarExchange (67) + : +- CometHashAggregate (66) + : +- CometHashAggregate (65) + : +- CometColumnarExchange (64) + : +- CometHashAggregate (63) + : +- CometUnion (62) + : :- CometProject (23) + : : +- CometSortMergeJoin (22) + : : :- CometSort (16) + : : : +- CometColumnarExchange (15) + : : : +- RowToColumnar (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- ReusedExchange (11) + : : +- CometSort (21) + : : +- CometColumnarExchange (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : :- CometProject (42) + : : +- CometSortMergeJoin (41) + : : :- CometSort (35) + : : : +- CometColumnarExchange (34) + : : : +- RowToColumnar (33) + : : : +- * Project (32) + : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : :- * Project (29) + : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : :- * ColumnarToRow (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- CometSort (40) + : : +- CometColumnarExchange (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.store_returns (36) + : +- CometProject (61) + : +- CometSortMergeJoin (60) + : :- CometSort (54) + : : +- CometColumnarExchange (53) + : : +- RowToColumnar (52) + : : +- * Project (51) + : : +- * BroadcastHashJoin Inner BuildRight (50) + : : :- * Project (48) + : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : :- * ColumnarToRow (45) + : : : : +- CometFilter (44) + : : : : +- CometScan parquet spark_catalog.default.web_sales (43) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- CometSort (59) + : +- CometColumnarExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.web_returns (55) + +- * ColumnarToRow (131) + +- CometSort (130) + +- CometColumnarExchange (129) + +- CometFilter (128) + +- CometHashAggregate (127) + +- CometColumnarExchange (126) + +- CometHashAggregate (125) + +- CometHashAggregate (124) + +- CometColumnarExchange (123) + +- CometHashAggregate (122) + +- CometUnion (121) + :- CometProject (88) + : +- CometSortMergeJoin (87) + : :- CometSort (84) + : : +- CometColumnarExchange (83) + : : +- RowToColumnar (82) + : : +- * Project (81) + : : +- * BroadcastHashJoin Inner BuildRight (80) + : : :- * Project (78) + : : : +- * BroadcastHashJoin Inner BuildRight (77) + : : : :- * ColumnarToRow (75) + : : : : +- CometFilter (74) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) + : : : +- ReusedExchange (76) + : : +- ReusedExchange (79) + : +- CometSort (86) + : +- ReusedExchange (85) + :- CometProject (104) + : +- CometSortMergeJoin (103) + : :- CometSort (100) + : : +- CometColumnarExchange (99) + : : +- RowToColumnar (98) + : : +- * Project (97) + : : +- * BroadcastHashJoin Inner BuildRight (96) + : : :- * Project (94) + : : : +- * BroadcastHashJoin Inner BuildRight (93) + : : : :- * ColumnarToRow (91) + : : : : +- CometFilter (90) + : : : : +- CometScan parquet spark_catalog.default.store_sales (89) + : : : +- ReusedExchange (92) + : : +- ReusedExchange (95) + : +- CometSort (102) + : +- ReusedExchange (101) + +- CometProject (120) + +- CometSortMergeJoin (119) + :- CometSort (116) + : +- CometColumnarExchange (115) + : +- RowToColumnar (114) + : +- * Project (113) + : +- * BroadcastHashJoin Inner BuildRight (112) + : :- * Project (110) + : : +- * BroadcastHashJoin Inner BuildRight (109) + : : :- * ColumnarToRow (107) + : : : +- CometFilter (106) + : : : +- CometScan parquet spark_catalog.default.web_sales (105) + : : +- ReusedExchange (108) + : +- ReusedExchange (111) + +- CometSort (118) + +- ReusedExchange (117) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -180,7 +182,7 @@ Join condition: None Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 136] +(11) ReusedExchange [Reuses operator id: 138] Output [2]: [d_date_sk#13, d_year#14] (12) BroadcastHashJoin [codegen id : 3] @@ -193,13 +195,16 @@ Join condition: None Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -(14) Exchange +(14) RowToColumnar Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) Sort [codegen id : 4] +(15) CometColumnarExchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometSort +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] @@ -208,34 +213,30 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(17) CometFilter +(18) CometFilter Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) -(18) CometProject +(19) CometProject Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(19) CometExchange +(20) CometColumnarExchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) CometSort +(21) CometSort Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] -(21) ColumnarToRow [codegen id : 5] -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +(22) CometSortMergeJoin +Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter -(22) SortMergeJoin [codegen id : 6] -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#16, cr_item_sk#15] -Join type: LeftOuter -Join condition: None - -(23) Project [codegen id : 6] -Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] +(23) CometProject Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] (unknown) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] @@ -249,42 +250,45 @@ ReadSchema: struct -(36) CometFilter +(37) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(37) CometProject +(38) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(38) CometExchange +(39) CometColumnarExchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(39) CometSort +(40) CometSort Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(40) ColumnarToRow [codegen id : 11] -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(41) SortMergeJoin [codegen id : 12] -Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] -Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] -Join type: LeftOuter -Join condition: None +(41) CometSortMergeJoin +Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(42) Project [codegen id : 12] -Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] +(42) CometProject Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] (unknown) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] @@ -334,42 +334,45 @@ ReadSchema: struct -(55) CometFilter +(56) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(56) CometProject +(57) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(57) CometExchange +(58) CometColumnarExchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(58) CometSort +(59) CometSort Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(59) ColumnarToRow [codegen id : 17] -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(60) SortMergeJoin [codegen id : 18] -Left keys [2]: [ws_order_number#43, ws_item_sk#42] -Right keys [2]: [wr_order_number#56, wr_item_sk#55] -Join type: LeftOuter -Join condition: None +(60) CometSortMergeJoin +Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(61) Project [codegen id : 18] -Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] +(61) CometProject Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] -(62) Union +(62) CometUnion +Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(63) HashAggregate [codegen id : 19] +(63) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(64) Exchange +(64) CometColumnarExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(65) HashAggregate [codegen id : 20] +(65) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(66) HashAggregate [codegen id : 20] +(66) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum#62, sum#63] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -(67) Exchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(67) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) HashAggregate [codegen id : 21] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] +(68) CometHashAggregate +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] -(69) Filter [codegen id : 21] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] -Condition : isnotnull(sales_cnt#68) +(69) CometFilter +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Condition : isnotnull(sales_cnt#64) -(70) Exchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(70) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(71) Sort [codegen id : 22] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] -Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 +(71) CometSort +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] + +(72) ColumnarToRow [codegen id : 10] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(73) CometFilter -Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] -Condition : isnotnull(cs_item_sk#70) +(74) CometFilter +Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Condition : isnotnull(cs_item_sk#66) -(74) ColumnarToRow [codegen id : 25] -Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +(75) ColumnarToRow [codegen id : 13] +Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -(75) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(76) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(76) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [cs_item_sk#70] -Right keys [1]: [i_item_sk#76] +(77) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_item_sk#66] +Right keys [1]: [i_item_sk#72] Join type: Inner Join condition: None -(77) Project [codegen id : 25] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(78) Project [codegen id : 13] +Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(78) ReusedExchange [Reuses operator id: 140] -Output [2]: [d_date_sk#81, d_year#82] +(79) ReusedExchange [Reuses operator id: 142] +Output [2]: [d_date_sk#77, d_year#78] -(79) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [cs_sold_date_sk#74] -Right keys [1]: [d_date_sk#81] +(80) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_sold_date_sk#70] +Right keys [1]: [d_date_sk#77] Join type: Inner Join condition: None -(80) Project [codegen id : 25] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] - -(81) Exchange -Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(81) Project [codegen id : 13] +Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -(82) Sort [codegen id : 26] -Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 +(82) RowToColumnar +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -(83) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +(83) CometColumnarExchange +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] (84) CometSort -Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -Arguments: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86], [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST] +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] -(85) ColumnarToRow [codegen id : 27] -Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +(85) ReusedExchange [Reuses operator id: 20] +Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -(86) SortMergeJoin [codegen id : 28] -Left keys [2]: [cs_order_number#71, cs_item_sk#70] -Right keys [2]: [cr_order_number#84, cr_item_sk#83] -Join type: LeftOuter -Join condition: None +(86) CometSort +Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] + +(87) CometSortMergeJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter -(87) Project [codegen id : 28] -Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#20, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#21] -Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +(88) CometProject +Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] (unknown) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(89) CometFilter -Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -Condition : isnotnull(ss_item_sk#87) +(90) CometFilter +Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Condition : isnotnull(ss_item_sk#83) -(90) ColumnarToRow [codegen id : 31] -Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +(91) ColumnarToRow [codegen id : 16] +Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -(91) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(92) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(92) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [ss_item_sk#87] -Right keys [1]: [i_item_sk#93] +(93) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ss_item_sk#83] +Right keys [1]: [i_item_sk#89] Join type: Inner Join condition: None -(93) Project [codegen id : 31] -Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(94) Project [codegen id : 16] +Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(94) ReusedExchange [Reuses operator id: 140] -Output [2]: [d_date_sk#98, d_year#99] +(95) ReusedExchange [Reuses operator id: 142] +Output [2]: [d_date_sk#94, d_year#95] -(95) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [ss_sold_date_sk#91] -Right keys [1]: [d_date_sk#98] +(96) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ss_sold_date_sk#87] +Right keys [1]: [d_date_sk#94] Join type: Inner Join condition: None -(96) Project [codegen id : 31] -Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] - -(97) Exchange -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(97) Project [codegen id : 16] +Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -(98) Sort [codegen id : 32] -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 +(98) RowToColumnar +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -(99) ReusedExchange [Reuses operator id: 38] -Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(99) CometColumnarExchange +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] (100) CometSort -Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103], [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST] +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] -(101) ColumnarToRow [codegen id : 33] -Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(101) ReusedExchange [Reuses operator id: 39] +Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -(102) SortMergeJoin [codegen id : 34] -Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] -Right keys [2]: [sr_ticket_number#101, sr_item_sk#100] -Join type: LeftOuter -Join condition: None +(102) CometSort +Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] + +(103) CometSortMergeJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter -(103) Project [codegen id : 34] -Output [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#40, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#41] -Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(104) CometProject +Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] (unknown) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] +PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(105) CometFilter -Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Condition : isnotnull(ws_item_sk#104) +(106) CometFilter +Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Condition : isnotnull(ws_item_sk#100) -(106) ColumnarToRow [codegen id : 37] -Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +(107) ColumnarToRow [codegen id : 19] +Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -(107) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +(108) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(108) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_item_sk#104] -Right keys [1]: [i_item_sk#110] +(109) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_item_sk#100] +Right keys [1]: [i_item_sk#106] Join type: Inner Join condition: None -(109) Project [codegen id : 37] -Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +(110) Project [codegen id : 19] +Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(110) ReusedExchange [Reuses operator id: 140] -Output [2]: [d_date_sk#115, d_year#116] +(111) ReusedExchange [Reuses operator id: 142] +Output [2]: [d_date_sk#111, d_year#112] -(111) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#108] -Right keys [1]: [d_date_sk#115] +(112) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_sold_date_sk#104] +Right keys [1]: [d_date_sk#111] Join type: Inner Join condition: None -(112) Project [codegen id : 37] -Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] - -(113) Exchange -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, [plan_id=13] +(113) Project [codegen id : 19] +Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -(114) Sort [codegen id : 38] -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST], false, 0 +(114) RowToColumnar +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -(115) ReusedExchange [Reuses operator id: 57] -Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +(115) CometColumnarExchange +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] (116) CometSort -Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] -(117) ColumnarToRow [codegen id : 39] -Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +(117) ReusedExchange [Reuses operator id: 58] +Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -(118) SortMergeJoin [codegen id : 40] -Left keys [2]: [ws_order_number#105, ws_item_sk#104] -Right keys [2]: [wr_order_number#118, wr_item_sk#117] -Join type: LeftOuter -Join condition: None +(118) CometSort +Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] + +(119) CometSortMergeJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter -(119) Project [codegen id : 40] -Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#60, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#61] -Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +(120) CometProject +Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] -(120) Union +(121) CometUnion +Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] -(121) HashAggregate [codegen id : 41] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +(122) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(122) Exchange -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(123) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(123) HashAggregate [codegen id : 42] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +(124) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(124) HashAggregate [codegen id : 42] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(125) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum#62, sum#121] -Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -(125) Exchange -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(126) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(126) HashAggregate [codegen id : 43] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(127) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] -Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#20)#66 AS sales_cnt#123, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#124] -(127) Filter [codegen id : 43] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -Condition : isnotnull(sales_cnt#123) +(128) CometFilter +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Condition : isnotnull(sales_cnt#118) + +(129) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(128) Exchange -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] +(130) CometSort +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] -(129) Sort [codegen id : 44] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 +(131) ColumnarToRow [codegen id : 20] +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(130) SortMergeJoin [codegen id : 45] +(132) SortMergeJoin [codegen id : 21] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Right keys [4]: [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Join type: Inner -Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) -(131) Project [codegen id : 45] -Output [10]: [d_year#82 AS prev_year#125, d_year#14 AS year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#123 AS prev_yr_cnt#127, sales_cnt#68 AS curr_yr_cnt#128, (sales_cnt#68 - sales_cnt#123) AS sales_cnt_diff#129, (sales_amt#69 - sales_amt#124) AS sales_amt_diff#130] -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] +(133) Project [codegen id : 21] +Output [10]: [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(132) TakeOrderedAndProject -Input [10]: [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] -Arguments: 100, [sales_cnt_diff#129 ASC NULLS FIRST], [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] +(134) TakeOrderedAndProject +Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] +Arguments: 100, [sales_cnt_diff#124 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (136) -+- * ColumnarToRow (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +BroadcastExchange (138) ++- * ColumnarToRow (137) + +- CometFilter (136) + +- CometScan parquet spark_catalog.default.date_dim (135) (unknown) Scan parquet spark_catalog.default.date_dim @@ -744,14 +736,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(134) CometFilter +(136) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(135) ColumnarToRow [codegen id : 1] +(137) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(136) BroadcastExchange +(138) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] @@ -759,33 +751,33 @@ Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 -BroadcastExchange (140) -+- * ColumnarToRow (139) - +- CometFilter (138) - +- CometScan parquet spark_catalog.default.date_dim (137) +Subquery:4 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (142) ++- * ColumnarToRow (141) + +- CometFilter (140) + +- CometScan parquet spark_catalog.default.date_dim (139) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#81, d_year#82] +Output [2]: [d_date_sk#77, d_year#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(138) CometFilter -Input [2]: [d_date_sk#81, d_year#82] -Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) +(140) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(139) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#81, d_year#82] +(141) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#77, d_year#78] -(140) BroadcastExchange -Input [2]: [d_date_sk#81, d_year#82] +(142) BroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 89 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 104 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt index 837398c8f1..153383e5d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt @@ -1,237 +1,185 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_amt_diff] - WholeStageCodegen (45) + WholeStageCodegen (21) Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter - WholeStageCodegen (22) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (10) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (21) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - WholeStageCodegen (20) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - WholeStageCodegen (19) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - WholeStageCodegen (6) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - InputAdapter - WholeStageCodegen (5) + CometSort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + CometFilter [sales_cnt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometSort [cs_order_number,cs_item_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + RowToColumnar + WholeStageCodegen (3) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [cr_order_number,cr_item_sk] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - WholeStageCodegen (12) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (10) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (9) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] InputAdapter - WholeStageCodegen (11) + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cr_order_number,cr_item_sk] + CometColumnarExchange [cr_order_number,cr_item_sk] #7 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 + RowToColumnar + WholeStageCodegen (6) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [sr_ticket_number,sr_item_sk] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - WholeStageCodegen (18) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - WholeStageCodegen (16) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (15) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter - WholeStageCodegen (17) + ReusedExchange [d_date_sk,d_year] #5 + CometSort [sr_ticket_number,sr_item_sk] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #9 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometSort [ws_order_number,ws_item_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #10 + RowToColumnar + WholeStageCodegen (9) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [wr_order_number,wr_item_sk] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [wr_order_number,wr_item_sk] + CometColumnarExchange [wr_order_number,wr_item_sk] #11 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter - WholeStageCodegen (44) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (20) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - WholeStageCodegen (43) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (42) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - WholeStageCodegen (41) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - WholeStageCodegen (28) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (26) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (25) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - InputAdapter - WholeStageCodegen (27) + CometSort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + CometFilter [sales_cnt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometSort [cs_order_number,cs_item_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #15 + RowToColumnar + WholeStageCodegen (13) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [cr_order_number,cr_item_sk] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - WholeStageCodegen (34) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (32) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (31) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter - WholeStageCodegen (33) + ReusedExchange [d_date_sk,d_year] #16 + CometSort [cr_order_number,cr_item_sk] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 + RowToColumnar + WholeStageCodegen (16) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - WholeStageCodegen (40) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter - WholeStageCodegen (38) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (37) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - InputAdapter - WholeStageCodegen (39) + ReusedExchange [d_date_sk,d_year] #16 + CometSort [sr_ticket_number,sr_item_sk] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometSort [ws_order_number,ws_item_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #18 + RowToColumnar + WholeStageCodegen (19) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [wr_order_number,wr_item_sk] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [wr_order_number,wr_item_sk] + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index 459bc5c019..3286597a34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -1,42 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (38) -+- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- Union (34) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (13) - : +- * ColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.date_dim (10) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.web_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * ColumnarToRow (27) - : : +- CometFilter (26) - : : +- CometScan parquet spark_catalog.default.catalog_sales (25) - : +- ReusedExchange (28) - +- ReusedExchange (31) +* ColumnarToRow (40) ++- CometTakeOrderedAndProject (39) + +- CometHashAggregate (38) + +- CometColumnarExchange (37) + +- RowToColumnar (36) + +- * HashAggregate (35) + +- Union (34) + :- * Project (15) + : +- * BroadcastHashJoin Inner BuildRight (14) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.item (4) + : +- BroadcastExchange (13) + : +- * ColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.date_dim (10) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.web_sales (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * ColumnarToRow (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.catalog_sales (25) + : +- ReusedExchange (28) + +- ReusedExchange (31) (unknown) Scan parquet spark_catalog.default.store_sales @@ -201,18 +203,22 @@ Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#12)) Aggregate Attributes [2]: [count#37, sum#38] Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] -(36) Exchange +(36) RowToColumnar Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] -Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(37) HashAggregate [codegen id : 11] +(37) CometColumnarExchange +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] +Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(38) CometHashAggregate Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] Keys [5]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] -Aggregate Attributes [2]: [count(1)#41, sum(UnscaledValue(ext_sales_price#12))#42] -Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count(1)#41 AS sales_cnt#43, MakeDecimal(sum(UnscaledValue(ext_sales_price#12))#42,17,2) AS sales_amt#44] -(38) TakeOrderedAndProject -Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] -Arguments: 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] +(39) CometTakeOrderedAndProject +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#41, sales_amt#42] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#10 ASC NULLS FIRST,col_name#11 ASC NULLS FIRST,d_year#8 ASC NULLS FIRST,d_qoy#9 ASC NULLS FIRST,i_category#6 ASC NULLS FIRST], output=[channel#10,col_name#11,d_year#8,d_qoy#9,i_category#6,sales_cnt#41,sales_amt#42]), 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#41, sales_amt#42] + +(40) ColumnarToRow [codegen id : 11] +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#41, sales_amt#42] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index 73e6b09afe..8cd2591c7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -1,58 +1,60 @@ -TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - WholeStageCodegen (11) - HashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] [count(1),sum(UnscaledValue(ext_sales_price)),sales_cnt,sales_amt,count,sum] - InputAdapter - Exchange [channel,col_name,d_year,d_qoy,i_category] #1 - WholeStageCodegen (10) - HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] - InputAdapter - Union - WholeStageCodegen (3) - Project [ss_store_sk,d_year,d_qoy,i_category,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] +WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] + CometColumnarExchange [channel,col_name,d_year,d_qoy,i_category] #1 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] + InputAdapter + Union + WholeStageCodegen (3) + Project [ss_store_sk,d_year,d_qoy,i_category,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + WholeStageCodegen (6) + Project [ws_ship_customer_sk,d_year,d_qoy,i_category,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_ship_customer_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - WholeStageCodegen (6) - Project [ws_ship_customer_sk,d_year,d_qoy,i_category,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_ship_customer_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedExchange [i_item_sk,i_category] #2 InputAdapter - ReusedExchange [i_item_sk,i_category] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - WholeStageCodegen (9) - Project [cs_ship_addr_sk,d_year,d_qoy,i_category,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_ship_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + WholeStageCodegen (9) + Project [cs_ship_addr_sk,d_year,d_qoy,i_category,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_ship_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + InputAdapter + ReusedExchange [i_item_sk,i_category] #2 InputAdapter - ReusedExchange [i_item_sk,i_category] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index 692db69b81..2b95b83f04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -1,89 +1,103 @@ == Physical Plan == -TakeOrderedAndProject (85) -+- * HashAggregate (84) - +- Exchange (83) - +- * HashAggregate (82) - +- * Expand (81) - +- Union (80) - :- * Project (30) - : +- * BroadcastHashJoin LeftOuter BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.store_returns (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - :- * Project (49) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (48) - : :- BroadcastExchange (39) - : : +- * HashAggregate (38) - : : +- Exchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * ColumnarToRow (32) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : +- ReusedExchange (33) - : +- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * ColumnarToRow (41) - : : +- CometScan parquet spark_catalog.default.catalog_returns (40) - : +- ReusedExchange (42) - +- * Project (79) - +- * BroadcastHashJoin LeftOuter BuildRight (78) - :- * HashAggregate (64) - : +- Exchange (63) - : +- * HashAggregate (62) - : +- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Project (55) - : : +- * BroadcastHashJoin Inner BuildRight (54) - : : :- * ColumnarToRow (52) - : : : +- CometFilter (51) - : : : +- CometScan parquet spark_catalog.default.web_sales (50) - : : +- ReusedExchange (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_page (56) - +- BroadcastExchange (77) - +- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * ColumnarToRow (67) - : : +- CometFilter (66) - : : +- CometScan parquet spark_catalog.default.web_returns (65) - : +- ReusedExchange (68) - +- ReusedExchange (71) +* ColumnarToRow (99) ++- CometTakeOrderedAndProject (98) + +- CometHashAggregate (97) + +- CometColumnarExchange (96) + +- RowToColumnar (95) + +- * HashAggregate (94) + +- * Expand (93) + +- Union (92) + :- * Project (34) + : +- * BroadcastHashJoin LeftOuter BuildRight (33) + : :- * ColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometColumnarExchange (15) + : : +- RowToColumnar (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometHashAggregate (30) + : +- CometColumnarExchange (29) + : +- RowToColumnar (28) + : +- * HashAggregate (27) + : +- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * ColumnarToRow (20) + : : : +- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.store_returns (18) + : : +- ReusedExchange (21) + : +- ReusedExchange (24) + :- * Project (57) + : +- * BroadcastNestedLoopJoin Inner BuildLeft (56) + : :- BroadcastExchange (45) + : : +- * ColumnarToRow (44) + : : +- CometHashAggregate (43) + : : +- CometColumnarExchange (42) + : : +- RowToColumnar (41) + : : +- * HashAggregate (40) + : : +- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * ColumnarToRow (36) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (35) + : : +- ReusedExchange (37) + : +- * ColumnarToRow (55) + : +- CometHashAggregate (54) + : +- CometColumnarExchange (53) + : +- RowToColumnar (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * ColumnarToRow (47) + : : +- CometScan parquet spark_catalog.default.catalog_returns (46) + : +- ReusedExchange (48) + +- * Project (91) + +- * BroadcastHashJoin LeftOuter BuildRight (90) + :- * ColumnarToRow (74) + : +- CometHashAggregate (73) + : +- CometColumnarExchange (72) + : +- RowToColumnar (71) + : +- * HashAggregate (70) + : +- * Project (69) + : +- * BroadcastHashJoin Inner BuildRight (68) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * ColumnarToRow (60) + : : : +- CometFilter (59) + : : : +- CometScan parquet spark_catalog.default.web_sales (58) + : : +- ReusedExchange (61) + : +- BroadcastExchange (67) + : +- * ColumnarToRow (66) + : +- CometFilter (65) + : +- CometScan parquet spark_catalog.default.web_page (64) + +- BroadcastExchange (89) + +- * ColumnarToRow (88) + +- CometHashAggregate (87) + +- CometColumnarExchange (86) + +- RowToColumnar (85) + +- * HashAggregate (84) + +- * Project (83) + +- * BroadcastHashJoin Inner BuildRight (82) + :- * Project (80) + : +- * BroadcastHashJoin Inner BuildRight (79) + : :- * ColumnarToRow (77) + : : +- CometFilter (76) + : : +- CometScan parquet spark_catalog.default.web_returns (75) + : +- ReusedExchange (78) + +- ReusedExchange (81) (unknown) Scan parquet spark_catalog.default.store_sales @@ -101,7 +115,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 3] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 90] +(4) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -149,399 +163,427 @@ Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(Un Aggregate Attributes [2]: [sum#8, sum#9] Results [3]: [s_store_sk#7, sum#10, sum#11] -(14) Exchange +(14) RowToColumnar Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 8] +(15) CometColumnarExchange +Input [3]: [s_store_sk#7, sum#10, sum#11] +Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] + +(17) ColumnarToRow [codegen id : 8] +Input [3]: [s_store_sk#7, sales#12, profit#13] (unknown) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Output [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(sr_returned_date_sk#17 IN dynamicpruning#18)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(17) CometFilter -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) +(19) CometFilter +Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] +Condition : isnotnull(sr_store_sk#14) -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +(20) ColumnarToRow [codegen id : 6] +Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] -(19) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#21] +(21) ReusedExchange [Reuses operator id: 104] +Output [1]: [d_date_sk#19] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#21] +(22) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_returned_date_sk#17] +Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(21) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] +(23) Project [codegen id : 6] +Output [3]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16] +Input [5]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17, d_date_sk#19] -(22) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#22] +(24) ReusedExchange [Reuses operator id: 10] +Output [1]: [s_store_sk#20] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#22] +(25) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_store_sk#14] +Right keys [1]: [s_store_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] - -(25) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Keys [1]: [s_store_sk#22] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#23, sum#24] -Results [3]: [s_store_sk#22, sum#25, sum#26] - -(26) Exchange -Input [3]: [s_store_sk#22, sum#25, sum#26] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#22, sum#25, sum#26] -Keys [1]: [s_store_sk#22] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] -Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] - -(28) BroadcastExchange -Input [3]: [s_store_sk#22, returns#29, profit_loss#30] +(26) Project [codegen id : 6] +Output [3]: [sr_return_amt#15, sr_net_loss#16, s_store_sk#20] +Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, s_store_sk#20] + +(27) HashAggregate [codegen id : 6] +Input [3]: [sr_return_amt#15, sr_net_loss#16, s_store_sk#20] +Keys [1]: [s_store_sk#20] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#15)), partial_sum(UnscaledValue(sr_net_loss#16))] +Aggregate Attributes [2]: [sum#21, sum#22] +Results [3]: [s_store_sk#20, sum#23, sum#24] + +(28) RowToColumnar +Input [3]: [s_store_sk#20, sum#23, sum#24] + +(29) CometColumnarExchange +Input [3]: [s_store_sk#20, sum#23, sum#24] +Arguments: hashpartitioning(s_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(30) CometHashAggregate +Input [3]: [s_store_sk#20, sum#23, sum#24] +Keys [1]: [s_store_sk#20] +Functions [2]: [sum(UnscaledValue(sr_return_amt#15)), sum(UnscaledValue(sr_net_loss#16))] + +(31) ColumnarToRow [codegen id : 7] +Input [3]: [s_store_sk#20, returns#25, profit_loss#26] + +(32) BroadcastExchange +Input [3]: [s_store_sk#20, returns#25, profit_loss#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 8] +(33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#22] +Right keys [1]: [s_store_sk#20] Join type: LeftOuter Join condition: None -(30) Project [codegen id : 8] -Output [5]: [sales#14, coalesce(returns#29, 0.00) AS returns#31, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#32, store channel AS channel#33, s_store_sk#7 AS id#34] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] +(34) Project [codegen id : 8] +Output [5]: [sales#12, coalesce(returns#25, 0.00) AS returns#27, (profit#13 - coalesce(profit_loss#26, 0.00)) AS profit#28, store channel AS channel#29, s_store_sk#7 AS id#30] +Input [6]: [s_store_sk#7, sales#12, profit#13, s_store_sk#20, returns#25, profit_loss#26] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +Output [4]: [cs_call_center_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#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(36) ColumnarToRow [codegen id : 10] +Input [4]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -(33) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#40] +(37) ReusedExchange [Reuses operator id: 104] +Output [1]: [d_date_sk#36] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#34] +Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(35) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] - -(36) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum#41, sum#42] -Results [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(37) Exchange -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(38) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] -Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] - -(39) BroadcastExchange -Input [3]: [cs_call_center_sk#35, sales#47, profit#48] +(39) Project [codegen id : 10] +Output [3]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33] +Input [5]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, d_date_sk#36] + +(40) HashAggregate [codegen id : 10] +Input [3]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33] +Keys [1]: [cs_call_center_sk#31] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(UnscaledValue(cs_net_profit#33))] +Aggregate Attributes [2]: [sum#37, sum#38] +Results [3]: [cs_call_center_sk#31, sum#39, sum#40] + +(41) RowToColumnar +Input [3]: [cs_call_center_sk#31, sum#39, sum#40] + +(42) CometColumnarExchange +Input [3]: [cs_call_center_sk#31, sum#39, sum#40] +Arguments: hashpartitioning(cs_call_center_sk#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(43) CometHashAggregate +Input [3]: [cs_call_center_sk#31, sum#39, sum#40] +Keys [1]: [cs_call_center_sk#31] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(UnscaledValue(cs_net_profit#33))] + +(44) ColumnarToRow [codegen id : 11] +Input [3]: [cs_call_center_sk#31, sales#41, profit#42] + +(45) BroadcastExchange +Input [3]: [cs_call_center_sk#31, sales#41, profit#42] Arguments: IdentityBroadcastMode, [plan_id=6] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Output [3]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#46)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(47) ColumnarToRow [codegen id : 13] +Input [3]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -(42) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#53] +(48) ReusedExchange [Reuses operator id: 104] +Output [1]: [d_date_sk#47] -(43) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#51] -Right keys [1]: [d_date_sk#53] +(49) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cr_returned_date_sk#45] +Right keys [1]: [d_date_sk#47] Join type: Inner Join condition: None -(44) Project [codegen id : 13] -Output [2]: [cr_return_amount#49, cr_net_loss#50] -Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] +(50) Project [codegen id : 13] +Output [2]: [cr_return_amount#43, cr_net_loss#44] +Input [4]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45, d_date_sk#47] -(45) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#49, cr_net_loss#50] +(51) HashAggregate [codegen id : 13] +Input [2]: [cr_return_amount#43, cr_net_loss#44] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum#54, sum#55] -Results [2]: [sum#56, sum#57] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#43)), partial_sum(UnscaledValue(cr_net_loss#44))] +Aggregate Attributes [2]: [sum#48, sum#49] +Results [2]: [sum#50, sum#51] + +(52) RowToColumnar +Input [2]: [sum#50, sum#51] -(46) Exchange -Input [2]: [sum#56, sum#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(53) CometColumnarExchange +Input [2]: [sum#50, sum#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(47) HashAggregate -Input [2]: [sum#56, sum#57] +(54) CometHashAggregate +Input [2]: [sum#50, sum#51] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] +Functions [2]: [sum(UnscaledValue(cr_return_amount#43)), sum(UnscaledValue(cr_net_loss#44))] -(48) BroadcastNestedLoopJoin [codegen id : 14] +(55) ColumnarToRow +Input [2]: [returns#52, profit_loss#53] + +(56) BroadcastNestedLoopJoin [codegen id : 14] Join type: Inner Join condition: None -(49) Project [codegen id : 14] -Output [5]: [sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#62, catalog channel AS channel#63, cs_call_center_sk#35 AS id#64] -Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] +(57) Project [codegen id : 14] +Output [5]: [sales#41, returns#52, (profit#42 - profit_loss#53) AS profit#54, catalog channel AS channel#55, cs_call_center_sk#31 AS id#56] +Input [5]: [cs_call_center_sk#31, sales#41, profit#42, returns#52, profit_loss#53] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +Output [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(51) CometFilter -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_web_page_sk#65) +(59) CometFilter +Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] +Condition : isnotnull(ws_web_page_sk#57) -(52) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(60) ColumnarToRow [codegen id : 17] +Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] -(53) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#70] +(61) ReusedExchange [Reuses operator id: 104] +Output [1]: [d_date_sk#62] -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#68] -Right keys [1]: [d_date_sk#70] +(62) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#60] +Right keys [1]: [d_date_sk#62] Join type: Inner Join condition: None -(55) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] -Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] +(63) Project [codegen id : 17] +Output [3]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59] +Input [5]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60, d_date_sk#62] (unknown) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#71] +Output [1]: [wp_web_page_sk#63] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(57) CometFilter -Input [1]: [wp_web_page_sk#71] -Condition : isnotnull(wp_web_page_sk#71) +(65) CometFilter +Input [1]: [wp_web_page_sk#63] +Condition : isnotnull(wp_web_page_sk#63) -(58) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#71] +(66) ColumnarToRow [codegen id : 16] +Input [1]: [wp_web_page_sk#63] -(59) BroadcastExchange -Input [1]: [wp_web_page_sk#71] +(67) BroadcastExchange +Input [1]: [wp_web_page_sk#63] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#65] -Right keys [1]: [wp_web_page_sk#71] +(68) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_web_page_sk#57] +Right keys [1]: [wp_web_page_sk#63] Join type: Inner Join condition: None -(61) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +(69) Project [codegen id : 17] +Output [3]: [ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] +Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] + +(70) HashAggregate [codegen id : 17] +Input [3]: [ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] +Keys [1]: [wp_web_page_sk#63] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#58)), partial_sum(UnscaledValue(ws_net_profit#59))] +Aggregate Attributes [2]: [sum#64, sum#65] +Results [3]: [wp_web_page_sk#63, sum#66, sum#67] -(62) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum#72, sum#73] -Results [3]: [wp_web_page_sk#71, sum#74, sum#75] +(71) RowToColumnar +Input [3]: [wp_web_page_sk#63, sum#66, sum#67] -(63) Exchange -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(72) CometColumnarExchange +Input [3]: [wp_web_page_sk#63, sum#66, sum#67] +Arguments: hashpartitioning(wp_web_page_sk#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(64) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] -Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] +(73) CometHashAggregate +Input [3]: [wp_web_page_sk#63, sum#66, sum#67] +Keys [1]: [wp_web_page_sk#63] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#58)), sum(UnscaledValue(ws_net_profit#59))] + +(74) ColumnarToRow [codegen id : 22] +Input [3]: [wp_web_page_sk#63, sales#68, profit#69] (unknown) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +Output [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(wr_returned_date_sk#73), dynamicpruningexpression(wr_returned_date_sk#73 IN dynamicpruning#74)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(66) CometFilter -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] -Condition : isnotnull(wr_web_page_sk#80) +(76) CometFilter +Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] +Condition : isnotnull(wr_web_page_sk#70) -(67) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(77) ColumnarToRow [codegen id : 20] +Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] -(68) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#85] +(78) ReusedExchange [Reuses operator id: 104] +Output [1]: [d_date_sk#75] -(69) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#83] -Right keys [1]: [d_date_sk#85] +(79) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_returned_date_sk#73] +Right keys [1]: [d_date_sk#75] Join type: Inner Join condition: None -(70) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] -Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] +(80) Project [codegen id : 20] +Output [3]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72] +Input [5]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73, d_date_sk#75] -(71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#86] +(81) ReusedExchange [Reuses operator id: 67] +Output [1]: [wp_web_page_sk#76] -(72) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#80] -Right keys [1]: [wp_web_page_sk#86] +(82) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_web_page_sk#70] +Right keys [1]: [wp_web_page_sk#76] Join type: Inner Join condition: None -(73) Project [codegen id : 20] -Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] - -(74) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum#87, sum#88] -Results [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(75) Exchange -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(76) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] -Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] - -(77) BroadcastExchange -Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] +(83) Project [codegen id : 20] +Output [3]: [wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] +Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] + +(84) HashAggregate [codegen id : 20] +Input [3]: [wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] +Keys [1]: [wp_web_page_sk#76] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#71)), partial_sum(UnscaledValue(wr_net_loss#72))] +Aggregate Attributes [2]: [sum#77, sum#78] +Results [3]: [wp_web_page_sk#76, sum#79, sum#80] + +(85) RowToColumnar +Input [3]: [wp_web_page_sk#76, sum#79, sum#80] + +(86) CometColumnarExchange +Input [3]: [wp_web_page_sk#76, sum#79, sum#80] +Arguments: hashpartitioning(wp_web_page_sk#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(87) CometHashAggregate +Input [3]: [wp_web_page_sk#76, sum#79, sum#80] +Keys [1]: [wp_web_page_sk#76] +Functions [2]: [sum(UnscaledValue(wr_return_amt#71)), sum(UnscaledValue(wr_net_loss#72))] + +(88) ColumnarToRow [codegen id : 21] +Input [3]: [wp_web_page_sk#76, returns#81, profit_loss#82] + +(89) BroadcastExchange +Input [3]: [wp_web_page_sk#76, returns#81, profit_loss#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(78) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#71] -Right keys [1]: [wp_web_page_sk#86] +(90) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [wp_web_page_sk#63] +Right keys [1]: [wp_web_page_sk#76] Join type: LeftOuter Join condition: None -(79) Project [codegen id : 22] -Output [5]: [sales#78, coalesce(returns#93, 0.00) AS returns#95, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#96, web channel AS channel#97, wp_web_page_sk#71 AS id#98] -Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] +(91) Project [codegen id : 22] +Output [5]: [sales#68, coalesce(returns#81, 0.00) AS returns#83, (profit#69 - coalesce(profit_loss#82, 0.00)) AS profit#84, web channel AS channel#85, wp_web_page_sk#63 AS id#86] +Input [6]: [wp_web_page_sk#63, sales#68, profit#69, wp_web_page_sk#76, returns#81, profit_loss#82] + +(92) Union + +(93) Expand [codegen id : 23] +Input [5]: [sales#12, returns#27, profit#28, channel#29, id#30] +Arguments: [[sales#12, returns#27, profit#28, channel#29, id#30, 0], [sales#12, returns#27, profit#28, channel#29, null, 1], [sales#12, returns#27, profit#28, null, null, 3]], [sales#12, returns#27, profit#28, channel#87, id#88, spark_grouping_id#89] -(80) Union +(94) HashAggregate [codegen id : 23] +Input [6]: [sales#12, returns#27, profit#28, channel#87, id#88, spark_grouping_id#89] +Keys [3]: [channel#87, id#88, spark_grouping_id#89] +Functions [3]: [partial_sum(sales#12), partial_sum(returns#27), partial_sum(profit#28)] +Aggregate Attributes [6]: [sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Results [9]: [channel#87, id#88, spark_grouping_id#89, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -(81) Expand [codegen id : 23] -Input [5]: [sales#14, returns#31, profit#32, channel#33, id#34] -Arguments: [[sales#14, returns#31, profit#32, channel#33, id#34, 0], [sales#14, returns#31, profit#32, channel#33, null, 1], [sales#14, returns#31, profit#32, null, null, 3]], [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] +(95) RowToColumnar +Input [9]: [channel#87, id#88, spark_grouping_id#89, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -(82) HashAggregate [codegen id : 23] -Input [6]: [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] -Keys [3]: [channel#99, id#100, spark_grouping_id#101] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#31), partial_sum(profit#32)] -Aggregate Attributes [6]: [sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] -Results [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +(96) CometColumnarExchange +Input [9]: [channel#87, id#88, spark_grouping_id#89, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Arguments: hashpartitioning(channel#87, id#88, spark_grouping_id#89, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(83) Exchange -Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Arguments: hashpartitioning(channel#99, id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(97) CometHashAggregate +Input [9]: [channel#87, id#88, spark_grouping_id#89, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Keys [3]: [channel#87, id#88, spark_grouping_id#89] +Functions [3]: [sum(sales#12), sum(returns#27), sum(profit#28)] -(84) HashAggregate [codegen id : 24] -Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Keys [3]: [channel#99, id#100, spark_grouping_id#101] -Functions [3]: [sum(sales#14), sum(returns#31), sum(profit#32)] -Aggregate Attributes [3]: [sum(sales#14)#114, sum(returns#31)#115, sum(profit#32)#116] -Results [5]: [channel#99, id#100, sum(sales#14)#114 AS sales#117, sum(returns#31)#115 AS returns#118, sum(profit#32)#116 AS profit#119] +(98) CometTakeOrderedAndProject +Input [5]: [channel#87, id#88, sales#102, returns#103, profit#104] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#87 ASC NULLS FIRST,id#88 ASC NULLS FIRST], output=[channel#87,id#88,sales#102,returns#103,profit#104]), 100, [channel#87 ASC NULLS FIRST, id#88 ASC NULLS FIRST], [channel#87, id#88, sales#102, returns#103, profit#104] -(85) TakeOrderedAndProject -Input [5]: [channel#99, id#100, sales#117, returns#118, profit#119] -Arguments: 100, [channel#99 ASC NULLS FIRST, id#100 ASC NULLS FIRST], [channel#99, id#100, sales#117, returns#118, profit#119] +(99) ColumnarToRow [codegen id : 24] +Input [5]: [channel#87, id#88, sales#102, returns#103, profit#104] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (90) -+- * ColumnarToRow (89) - +- CometProject (88) - +- CometFilter (87) - +- CometScan parquet spark_catalog.default.date_dim (86) +BroadcastExchange (104) ++- * ColumnarToRow (103) + +- CometProject (102) + +- CometFilter (101) + +- CometScan parquet spark_catalog.default.date_dim (100) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#120] +Output [2]: [d_date_sk#6, d_date#105] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter -Input [2]: [d_date_sk#6, d_date#120] -Condition : (((isnotnull(d_date#120) AND (d_date#120 >= 2000-08-03)) AND (d_date#120 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) +(101) CometFilter +Input [2]: [d_date_sk#6, d_date#105] +Condition : (((isnotnull(d_date#105) AND (d_date#105 >= 2000-08-03)) AND (d_date#105 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) -(88) CometProject -Input [2]: [d_date_sk#6, d_date#120] +(102) CometProject +Input [2]: [d_date_sk#6, d_date#105] Arguments: [d_date_sk#6], [d_date_sk#6] -(89) ColumnarToRow [codegen id : 1] +(103) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(90) BroadcastExchange +(104) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 40 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 46 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 65 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 75 Hosting Expression = wr_returned_date_sk#73 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index d6693067f0..d8ada5be65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -1,143 +1,157 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (24) - 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] - InputAdapter - Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (23) - 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 (8) - Project [sales,returns,profit,profit_loss,s_store_sk] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - InputAdapter - Exchange [s_store_sk] #2 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan 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) +WholeStageCodegen (24) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (23) + 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 (8) + Project [sales,returns,profit,profit_loss,s_store_sk] + BroadcastHashJoin [s_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_sk,sum,sum] + CometColumnarExchange [s_store_sk] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan 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) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_sk,sum,sum] + CometColumnarExchange [s_store_sk] #6 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - InputAdapter - Exchange [s_store_sk] #6 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [sr_store_sk] - CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + CometFilter [sr_store_sk] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [s_store_sk] #4 + WholeStageCodegen (14) + Project [sales,returns,profit,profit_loss,cs_call_center_sk] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometHashAggregate [cs_call_center_sk,sum,sum] + CometColumnarExchange [cs_call_center_sk] #8 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + CometHashAggregate [sum,sum] + CometColumnarExchange #9 + RowToColumnar + WholeStageCodegen (13) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [s_store_sk] #4 - WholeStageCodegen (14) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - InputAdapter - Exchange [cs_call_center_sk] #8 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - InputAdapter - Exchange #9 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) - Project [sales,returns,profit,profit_loss,wp_web_page_sk] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - InputAdapter - Exchange [wp_web_page_sk] #10 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_web_page_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (16) - ColumnarToRow + CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - InputAdapter - Exchange [wp_web_page_sk] #13 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (22) + Project [sales,returns,profit,profit_loss,wp_web_page_sk] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + ColumnarToRow + InputAdapter + CometHashAggregate [wp_web_page_sk,sum,sum] + CometColumnarExchange [wp_web_page_sk] #10 + RowToColumnar + WholeStageCodegen (17) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - CometFilter [wr_web_page_sk] - CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [wp_web_page_sk] #11 + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (21) + ColumnarToRow + InputAdapter + CometHashAggregate [wp_web_page_sk,sum,sum] + CometColumnarExchange [wp_web_page_sk] #13 + RowToColumnar + WholeStageCodegen (20) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_web_page_sk] + CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [wp_web_page_sk] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index caab92168f..79c2b2a5bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -1,71 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * SortMergeJoin Inner (65) - :- * Project (43) - : +- * SortMergeJoin Inner (42) - : :- * Sort (20) - : : +- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (14) - : +- * Sort (41) - : +- * Filter (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometSortMergeJoin (30) - : : :- CometSort (24) - : : : +- CometExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan parquet spark_catalog.default.web_sales (21) - : : +- CometSort (29) - : : +- CometExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan parquet spark_catalog.default.web_returns (25) - : +- ReusedExchange (34) - +- * Sort (64) - +- * Filter (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * ColumnarToRow (56) - : +- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (57) +TakeOrderedAndProject (71) ++- * Project (70) + +- * ColumnarToRow (69) + +- CometSortMergeJoin (68) + :- CometProject (45) + : +- CometSortMergeJoin (44) + : :- CometSort (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) + : : +- RowToColumnar (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * ColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : +- ReusedExchange (14) + : +- CometSort (43) + : +- CometFilter (42) + : +- CometHashAggregate (41) + : +- CometColumnarExchange (40) + : +- RowToColumnar (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * ColumnarToRow (34) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometSortMergeJoin (31) + : : :- CometSort (25) + : : : +- CometColumnarExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.web_returns (26) + : +- ReusedExchange (35) + +- CometSort (67) + +- CometFilter (66) + +- CometHashAggregate (65) + +- CometColumnarExchange (64) + +- RowToColumnar (63) + +- * HashAggregate (62) + +- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * ColumnarToRow (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometColumnarExchange (48) + : : +- CometFilter (47) + : : +- CometScan parquet spark_catalog.default.catalog_sales (46) + : +- CometSort (54) + : +- CometColumnarExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.catalog_returns (50) + +- ReusedExchange (59) (unknown) Scan parquet spark_catalog.default.store_sales @@ -80,9 +84,9 @@ ReadSchema: struct -(22) CometFilter -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) +(23) CometFilter +Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +Condition : (isnotnull(ws_item_sk#24) AND isnotnull(ws_bill_customer_sk#25)) -(23) CometExchange -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(24) CometColumnarExchange +Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +Arguments: hashpartitioning(ws_order_number#26, ws_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometSort -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] +(25) CometSort +Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_order_number#26 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Output [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(26) CometFilter -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) +(27) CometFilter +Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] +Condition : (isnotnull(wr_order_number#33) AND isnotnull(wr_item_sk#32)) -(27) CometProject -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] +(28) CometProject +Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] +Arguments: [wr_item_sk#32, wr_order_number#33], [wr_item_sk#32, wr_order_number#33] -(28) CometExchange -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(29) CometColumnarExchange +Input [2]: [wr_item_sk#32, wr_order_number#33] +Arguments: hashpartitioning(wr_order_number#33, wr_item_sk#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(29) CometSort -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] +(30) CometSort +Input [2]: [wr_item_sk#32, wr_order_number#33] +Arguments: [wr_item_sk#32, wr_order_number#33], [wr_order_number#33 ASC NULLS FIRST, wr_item_sk#32 ASC NULLS FIRST] -(30) CometSortMergeJoin -Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Right output [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter +(31) CometSortMergeJoin +Left output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +Right output [2]: [wr_item_sk#32, wr_order_number#33] +Arguments: [ws_order_number#26, ws_item_sk#24], [wr_order_number#33, wr_item_sk#32], LeftOuter -(31) CometFilter -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Condition : isnull(wr_order_number#36) +(32) CometFilter +Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] +Condition : isnull(wr_order_number#33) -(32) CometProject -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +(33) CometProject +Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] +Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -(33) ColumnarToRow [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +(34) ColumnarToRow [codegen id : 4] +Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -(34) ReusedExchange [Reuses operator id: 71] -Output [2]: [d_date_sk#38, d_year#39] +(35) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#35, d_year#36] -(35) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] +(36) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#35] Join type: Inner Join condition: None -(36) Project [codegen id : 5] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] - -(37) HashAggregate [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum#40, sum#41, sum#42] -Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(38) Exchange -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(39) HashAggregate [codegen id : 6] -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] -Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] - -(40) Filter [codegen id : 6] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Condition : (coalesce(ws_qty#51, 0) > 0) - -(41) Sort [codegen id : 6] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 - -(42) SortMergeJoin [codegen id : 7] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] -Join type: Inner -Join condition: None +(37) Project [codegen id : 4] +Output [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] +Input [8]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, d_date_sk#35, d_year#36] + +(38) HashAggregate [codegen id : 4] +Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] +Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] +Functions [3]: [partial_sum(ws_quantity#27), partial_sum(UnscaledValue(ws_wholesale_cost#28)), partial_sum(UnscaledValue(ws_sales_price#29))] +Aggregate Attributes [3]: [sum#37, sum#38, sum#39] +Results [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] + +(39) RowToColumnar +Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] + +(40) CometColumnarExchange +Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] +Arguments: hashpartitioning(d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(41) CometHashAggregate +Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] +Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] +Functions [3]: [sum(ws_quantity#27), sum(UnscaledValue(ws_wholesale_cost#28)), sum(UnscaledValue(ws_sales_price#29))] + +(42) CometFilter +Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] +Condition : (coalesce(ws_qty#45, 0) > 0) + +(43) CometSort +Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] +Arguments: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47], [ws_sold_year#43 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST, ws_customer_sk#44 ASC NULLS FIRST] + +(44) CometSortMergeJoin +Left output [6]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] +Right output [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] +Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44], Inner -(43) Project [codegen id : 7] -Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] -Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +(45) CometProject +Input [12]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] +Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47], [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#54), dynamicpruningexpression(cs_sold_date_sk#54 IN dynamicpruning#55)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(45) CometFilter -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) +(47) CometFilter +Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +Condition : (isnotnull(cs_item_sk#49) AND isnotnull(cs_bill_customer_sk#48)) -(46) CometExchange -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(48) CometColumnarExchange +Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +Arguments: hashpartitioning(cs_order_number#50, cs_item_sk#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(47) CometSort -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] +(49) CometSort +Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_order_number#50 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Output [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(49) CometFilter -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) +(51) CometFilter +Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] +Condition : (isnotnull(cr_order_number#57) AND isnotnull(cr_item_sk#56)) -(50) CometProject -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] +(52) CometProject +Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] +Arguments: [cr_item_sk#56, cr_order_number#57], [cr_item_sk#56, cr_order_number#57] -(51) CometExchange -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(53) CometColumnarExchange +Input [2]: [cr_item_sk#56, cr_order_number#57] +Arguments: hashpartitioning(cr_order_number#57, cr_item_sk#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(52) CometSort -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] +(54) CometSort +Input [2]: [cr_item_sk#56, cr_order_number#57] +Arguments: [cr_item_sk#56, cr_order_number#57], [cr_order_number#57 ASC NULLS FIRST, cr_item_sk#56 ASC NULLS FIRST] -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Right output [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter +(55) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +Right output [2]: [cr_item_sk#56, cr_order_number#57] +Arguments: [cs_order_number#50, cs_item_sk#49], [cr_order_number#57, cr_item_sk#56], LeftOuter -(54) CometFilter -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Condition : isnull(cr_order_number#63) +(56) CometFilter +Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] +Condition : isnull(cr_order_number#57) -(55) CometProject -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(57) CometProject +Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] +Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -(56) ColumnarToRow [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(58) ColumnarToRow [codegen id : 6] +Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -(57) ReusedExchange [Reuses operator id: 71] -Output [2]: [d_date_sk#65, d_year#66] +(59) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#59, d_year#60] -(58) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#65] +(60) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#54] +Right keys [1]: [d_date_sk#59] Join type: Inner Join condition: None -(59) Project [codegen id : 9] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] - -(60) HashAggregate [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum#67, sum#68, sum#69] -Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] - -(61) Exchange -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(62) HashAggregate [codegen id : 10] -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] -Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] - -(63) Filter [codegen id : 10] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Condition : (coalesce(cs_qty#78, 0) > 0) - -(64) Sort [codegen id : 10] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 - -(65) SortMergeJoin [codegen id : 11] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] -Join type: Inner -Join condition: None +(61) Project [codegen id : 6] +Output [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] +Input [8]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, d_date_sk#59, d_year#60] + +(62) HashAggregate [codegen id : 6] +Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] +Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] +Functions [3]: [partial_sum(cs_quantity#51), partial_sum(UnscaledValue(cs_wholesale_cost#52)), partial_sum(UnscaledValue(cs_sales_price#53))] +Aggregate Attributes [3]: [sum#61, sum#62, sum#63] +Results [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] + +(63) RowToColumnar +Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] + +(64) CometColumnarExchange +Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] +Arguments: hashpartitioning(d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(65) CometHashAggregate +Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] +Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] +Functions [3]: [sum(cs_quantity#51), sum(UnscaledValue(cs_wholesale_cost#52)), sum(UnscaledValue(cs_sales_price#53))] + +(66) CometFilter +Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] +Condition : (coalesce(cs_qty#69, 0) > 0) + +(67) CometSort +Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] +Arguments: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71], [cs_sold_year#67 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST, cs_customer_sk#68 ASC NULLS FIRST] + +(68) CometSortMergeJoin +Left output [9]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] +Right output [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] +Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68], Inner + +(69) ColumnarToRow [codegen id : 7] +Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -(66) Project [codegen id : 11] -Output [12]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] -Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +(70) Project [codegen id : 7] +Output [12]: [round((cast(ss_qty#21 as double) / cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)), 2) AS ratio#72, ss_qty#21 AS store_qty#73, ss_wc#22 AS store_wholesale_cost#74, ss_sp#23 AS store_sales_price#75, (coalesce(ws_qty#45, 0) + coalesce(cs_qty#69, 0)) AS other_chan_qty#76, (coalesce(ws_wc#46, 0.00) + coalesce(cs_wc#70, 0.00)) AS other_chan_wholesale_cost#77, (coalesce(ws_sp#47, 0.00) + coalesce(cs_sp#71, 0.00)) AS other_chan_sales_price#78, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, cs_qty#69] +Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -(67) TakeOrderedAndProject -Input [12]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] -Arguments: 100, [ratio#81 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] +(71) TakeOrderedAndProject +Input [12]: [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, cs_qty#69] +Arguments: 100, [ratio#72 ASC NULLS FIRST, ss_qty#21 DESC NULLS LAST, ss_wc#22 DESC NULLS LAST, ss_sp#23 DESC NULLS LAST, other_chan_qty#76 ASC NULLS FIRST, other_chan_wholesale_cost#77 ASC NULLS FIRST, other_chan_sales_price#78 ASC NULLS FIRST, round((cast(ss_qty#21 as double) / cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)), 2) ASC NULLS FIRST], [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * ColumnarToRow (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim @@ -398,19 +406,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter +(73) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(71) BroadcastExchange +(75) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 21 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#54 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index 7b351afcd3..002e6d86d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -1,80 +1,74 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (11) + WholeStageCodegen (7) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (7) - Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_sold_year,ss_item_sk,ss_customer_sk] - HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (2) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [sr_ticket_number] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_ticket_number,sr_item_sk] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] + CometSort [ss_sold_year,ss_item_sk,ss_customer_sk] + CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [sr_ticket_number] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_ticket_number,sr_item_sk] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + CometSort [ws_sold_year,ws_item_sk,ws_customer_sk] + CometFilter [ws_qty] + CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (6) - Sort [ws_sold_year,ws_item_sk,ws_customer_sk] - Filter [ws_qty] - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (5) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [wr_order_number] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometExchange [ws_order_number,ws_item_sk] #6 - CometFilter [ws_item_sk,ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_order_number,wr_item_sk] - CometExchange [wr_order_number,wr_item_sk] #7 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (10) - Sort [cs_sold_year,cs_item_sk,cs_customer_sk] - Filter [cs_qty] - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (9) + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [wr_order_number] + CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometSort [ws_order_number,ws_item_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #6 + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_order_number,wr_item_sk] + CometColumnarExchange [wr_order_number,wr_item_sk] #7 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + CometSort [cs_sold_year,cs_item_sk,cs_customer_sk] + CometFilter [cs_qty] + CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 + RowToColumnar + WholeStageCodegen (6) HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -84,12 +78,12 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometFilter [cr_order_number] CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] CometSort [cs_order_number,cs_item_sk] - CometExchange [cs_order_number,cs_item_sk] #9 + CometColumnarExchange [cs_order_number,cs_item_sk] #9 CometFilter [cs_item_sk,cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometSort [cr_order_number,cr_item_sk] - CometExchange [cr_order_number,cr_item_sk] #10 + CometColumnarExchange [cr_order_number,cr_item_sk] #10 CometProject [cr_item_sk,cr_order_number] CometFilter [cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt index 667c05e1c6..344150578b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt @@ -1,34 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * BroadcastHashJoin Inner BuildRight (28) - :- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (27) - +- * ColumnarToRow (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.customer (24) +TakeOrderedAndProject (32) ++- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * ColumnarToRow (25) + : +- CometHashAggregate (24) + : +- CometColumnarExchange (23) + : +- RowToColumnar (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.customer (26) (unknown) Scan parquet spark_catalog.default.store_sales @@ -46,7 +48,7 @@ Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnu (3) ColumnarToRow [codegen id : 4] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -(4) ReusedExchange [Reuses operator id: 35] +(4) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#10] (5) BroadcastHashJoin [codegen id : 4] @@ -130,78 +132,82 @@ Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(Unscale Aggregate Attributes [2]: [sum#17, sum#18] Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -(22) Exchange +(22) RowToColumnar Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(23) HashAggregate [codegen id : 6] +(23) CometColumnarExchange +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(24) CometHashAggregate Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#21, sum(UnscaledValue(ss_net_profit#7))#22] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#21,17,2) AS amt#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#22,17,2) AS profit#24] + +(25) ColumnarToRow [codegen id : 6] +Input [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#21, profit#22] (unknown) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +Output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(25) CometFilter -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Condition : isnotnull(c_customer_sk#25) +(27) CometFilter +Input [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] +Condition : isnotnull(c_customer_sk#23) -(26) ColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +(28) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] -(27) BroadcastExchange -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +(29) BroadcastExchange +Input [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(28) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#25] +Right keys [1]: [c_customer_sk#23] Join type: Inner Join condition: None -(29) Project [codegen id : 6] -Output [7]: [c_last_name#27, c_first_name#26, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#26, c_last_name#27] +(31) Project [codegen id : 6] +Output [7]: [c_last_name#25, c_first_name#24, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#21, profit#22, s_city#13] +Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#21, profit#22, c_customer_sk#23, c_first_name#24, c_last_name#25] -(30) TakeOrderedAndProject -Input [7]: [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Arguments: 100, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24] +(32) TakeOrderedAndProject +Input [7]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#21, profit#22, s_city#13] +Arguments: 100, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#22 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#21, profit#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#29, d_dow#30] +Output [3]: [d_date_sk#10, d_year#27, d_dow#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [d_date_sk#10, d_year#29, d_dow#30] -Condition : (((isnotnull(d_dow#30) AND (d_dow#30 = 1)) AND d_year#29 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) +(34) CometFilter +Input [3]: [d_date_sk#10, d_year#27, d_dow#28] +Condition : (((isnotnull(d_dow#28) AND (d_dow#28 = 1)) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(33) CometProject -Input [3]: [d_date_sk#10, d_year#29, d_dow#30] +(35) CometProject +Input [3]: [d_date_sk#10, d_year#27, d_dow#28] Arguments: [d_date_sk#10], [d_date_sk#10] -(34) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(35) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt index 4c05c449c5..b86117fc50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt @@ -2,47 +2,49 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, WholeStageCodegen (6) Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] + ColumnarToRow InputAdapter - Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] + CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dow,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,s_city] + CometFilter [s_number_employees,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index f54999ff58..67656a9098 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -1,47 +1,53 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (37) - +- * HashAggregate (36) - +- Exchange (35) - +- * HashAggregate (34) - +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * ColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.customer_address (13) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * ColumnarToRow (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.customer_address (17) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan parquet spark_catalog.default.customer (20) +* ColumnarToRow (49) ++- CometTakeOrderedAndProject (48) + +- CometHashAggregate (47) + +- CometColumnarExchange (46) + +- RowToColumnar (45) + +- * HashAggregate (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.store (7) + +- BroadcastExchange (41) + +- * ColumnarToRow (40) + +- CometHashAggregate (39) + +- CometColumnarExchange (38) + +- RowToColumnar (37) + +- * HashAggregate (36) + +- * BroadcastHashJoin LeftSemi BuildRight (35) + :- * ColumnarToRow (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.customer_address (13) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometHashAggregate (30) + +- CometColumnarExchange (29) + +- RowToColumnar (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * ColumnarToRow (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.customer_address (17) + +- BroadcastExchange (24) + +- * ColumnarToRow (23) + +- CometProject (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.customer (20) (unknown) Scan parquet spark_catalog.default.store_sales @@ -59,7 +65,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 8] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 48] +(4) ReusedExchange [Reuses operator id: 54] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 8] @@ -170,118 +176,130 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] Results [2]: [ca_zip#12, count#16] -(28) Exchange +(28) RowToColumnar Input [2]: [ca_zip#12, count#16] -Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(29) HashAggregate [codegen id : 5] +(29) CometColumnarExchange +Input [2]: [ca_zip#12, count#16] +Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(30) CometHashAggregate Input [2]: [ca_zip#12, count#16] Keys [1]: [ca_zip#12] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [2]: [substr(ca_zip#12, 1, 5) AS ca_zip#18, count(1)#17 AS cnt#19] -(30) Filter [codegen id : 5] -Input [2]: [ca_zip#18, cnt#19] -Condition : (cnt#19 > 10) +(31) CometFilter +Input [2]: [ca_zip#17, cnt#18] +Condition : (cnt#18 > 10) + +(32) CometProject +Input [2]: [ca_zip#17, cnt#18] +Arguments: [ca_zip#17], [ca_zip#17] -(31) Project [codegen id : 5] -Output [1]: [ca_zip#18] -Input [2]: [ca_zip#18, cnt#19] +(33) ColumnarToRow [codegen id : 5] +Input [1]: [ca_zip#17] -(32) BroadcastExchange -Input [1]: [ca_zip#18] +(34) BroadcastExchange +Input [1]: [ca_zip#17] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 6] +(35) BroadcastHashJoin [codegen id : 6] Left keys [2]: [coalesce(ca_zip#10, ), isnull(ca_zip#10)] -Right keys [2]: [coalesce(ca_zip#18, ), isnull(ca_zip#18)] +Right keys [2]: [coalesce(ca_zip#17, ), isnull(ca_zip#17)] Join type: LeftSemi Join condition: None -(34) HashAggregate [codegen id : 6] +(36) HashAggregate [codegen id : 6] Input [1]: [ca_zip#10] Keys [1]: [ca_zip#10] Functions: [] Aggregate Attributes: [] Results [1]: [ca_zip#10] -(35) Exchange +(37) RowToColumnar Input [1]: [ca_zip#10] -Arguments: hashpartitioning(ca_zip#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(36) HashAggregate [codegen id : 7] +(38) CometColumnarExchange +Input [1]: [ca_zip#10] +Arguments: hashpartitioning(ca_zip#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(39) CometHashAggregate Input [1]: [ca_zip#10] Keys [1]: [ca_zip#10] Functions: [] -Aggregate Attributes: [] -Results [1]: [ca_zip#10] -(37) BroadcastExchange +(40) ColumnarToRow [codegen id : 7] +Input [1]: [ca_zip#10] + +(41) BroadcastExchange Input [1]: [ca_zip#10] Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=6] -(38) BroadcastHashJoin [codegen id : 8] +(42) BroadcastHashJoin [codegen id : 8] Left keys [1]: [substr(s_zip#8, 1, 2)] Right keys [1]: [substr(ca_zip#10, 1, 2)] Join type: Inner Join condition: None -(39) Project [codegen id : 8] +(43) Project [codegen id : 8] Output [2]: [ss_net_profit#2, s_store_name#7] Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#10] -(40) HashAggregate [codegen id : 8] +(44) HashAggregate [codegen id : 8] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#20] -Results [2]: [s_store_name#7, sum#21] +Aggregate Attributes [1]: [sum#19] +Results [2]: [s_store_name#7, sum#20] -(41) Exchange -Input [2]: [s_store_name#7, sum#21] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(45) RowToColumnar +Input [2]: [s_store_name#7, sum#20] -(42) HashAggregate [codegen id : 9] -Input [2]: [s_store_name#7, sum#21] +(46) CometColumnarExchange +Input [2]: [s_store_name#7, sum#20] +Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(47) CometHashAggregate +Input [2]: [s_store_name#7, sum#20] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS sum(ss_net_profit)#23] -(43) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#23] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#23] +(48) CometTakeOrderedAndProject +Input [2]: [s_store_name#7, sum(ss_net_profit)#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#7 ASC NULLS FIRST], output=[s_store_name#7,sum(ss_net_profit)#21]), 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#21] + +(49) ColumnarToRow [codegen id : 9] +Input [2]: [s_store_name#7, sum(ss_net_profit)#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#24, d_qoy#25] +Output [3]: [d_date_sk#5, d_year#22, d_qoy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2)) AND (d_year#24 = 1998)) AND isnotnull(d_date_sk#5)) +(51) CometFilter +Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] +Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 1998)) AND isnotnull(d_date_sk#5)) -(46) CometProject -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] +(52) CometProject +Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] Arguments: [d_date_sk#5], [d_date_sk#5] -(47) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(48) BroadcastExchange +(54) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index 76fa276931..f4b5da283b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -1,72 +1,78 @@ -TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (9) - HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - InputAdapter - Exchange [s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [s_store_name,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_store_name] - BroadcastHashJoin [s_zip,ca_zip] - Project [ss_net_profit,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] +WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] + CometHashAggregate [s_store_name,sum] + CometColumnarExchange [s_store_name] #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [s_store_name,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_store_name] + BroadcastHashJoin [s_zip,ca_zip] + Project [ss_net_profit,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [ca_zip] - InputAdapter - Exchange [ca_zip] #5 - WholeStageCodegen (6) - HashAggregate [ca_zip] - BroadcastHashJoin [ca_zip,ca_zip] - ColumnarToRow - InputAdapter - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_zip] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [ca_zip] - Filter [cnt] - HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] - InputAdapter - Exchange [ca_zip] #7 - WholeStageCodegen (4) - HashAggregate [ca_zip] [count,count] - Project [ca_zip] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] + CometHashAggregate [ca_zip] + CometColumnarExchange [ca_zip] #5 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ca_zip] + BroadcastHashJoin [ca_zip,ca_zip] + ColumnarToRow + InputAdapter + CometProject [ca_zip] [ca_zip] + CometFilter [ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_zip] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [ca_zip] + CometFilter [cnt] + CometHashAggregate [ca_zip,count] + CometColumnarExchange [ca_zip] #7 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ca_zip] [count,count] + Project [ca_zip] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [c_current_addr_sk] + CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] 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 dfd123a1d9..e08d527aa0 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,108 +1,116 @@ == Physical Plan == -TakeOrderedAndProject (104) -+- * HashAggregate (103) - +- Exchange (102) - +- * HashAggregate (101) - +- * Expand (100) - +- Union (99) - :- * HashAggregate (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * ColumnarToRow (12) - : : : : : +- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : : : +- ReusedExchange (13) - : : : +- BroadcastExchange (19) - : : : +- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.store (16) - : : +- BroadcastExchange (26) - : : +- * ColumnarToRow (25) - : : +- CometProject (24) - : : +- CometFilter (23) - : : +- CometScan parquet spark_catalog.default.item (22) - : +- BroadcastExchange (33) - : +- * ColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan parquet spark_catalog.default.promotion (29) - :- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * Project (53) - : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : :- * ColumnarToRow (50) - : : : : : +- CometProject (49) - : : : : : +- CometSortMergeJoin (48) - : : : : : :- CometSort (42) - : : : : : : +- CometExchange (41) - : : : : : : +- CometFilter (40) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) - : : : : : +- CometSort (47) - : : : : : +- CometExchange (46) - : : : : : +- CometProject (45) - : : : : : +- CometFilter (44) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (43) - : : : : +- ReusedExchange (51) - : : : +- BroadcastExchange (57) - : : : +- * ColumnarToRow (56) - : : : +- CometFilter (55) - : : : +- CometScan parquet spark_catalog.default.catalog_page (54) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- * HashAggregate (98) - +- Exchange (97) - +- * HashAggregate (96) - +- * Project (95) - +- * BroadcastHashJoin Inner BuildRight (94) - :- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * Project (89) - : : +- * BroadcastHashJoin Inner BuildRight (88) - : : :- * Project (83) - : : : +- * BroadcastHashJoin Inner BuildRight (82) - : : : :- * ColumnarToRow (80) - : : : : +- CometProject (79) - : : : : +- CometSortMergeJoin (78) - : : : : :- CometSort (72) - : : : : : +- CometExchange (71) - : : : : : +- CometFilter (70) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (69) - : : : : +- CometSort (77) - : : : : +- CometExchange (76) - : : : : +- CometProject (75) - : : : : +- CometFilter (74) - : : : : +- CometScan parquet spark_catalog.default.web_returns (73) - : : : +- ReusedExchange (81) - : : +- BroadcastExchange (87) - : : +- * ColumnarToRow (86) - : : +- CometFilter (85) - : : +- CometScan parquet spark_catalog.default.web_site (84) - : +- ReusedExchange (90) - +- ReusedExchange (93) +* ColumnarToRow (112) ++- CometTakeOrderedAndProject (111) + +- CometHashAggregate (110) + +- CometColumnarExchange (109) + +- RowToColumnar (108) + +- * HashAggregate (107) + +- * Expand (106) + +- Union (105) + :- * HashAggregate (40) + : +- * ColumnarToRow (39) + : +- CometColumnarExchange (38) + : +- RowToColumnar (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (15) + : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : :- * ColumnarToRow (12) + : : : : : +- CometProject (11) + : : : : : +- CometSortMergeJoin (10) + : : : : : :- CometSort (4) + : : : : : : +- CometColumnarExchange (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- CometSort (9) + : : : : : +- CometColumnarExchange (8) + : : : : : +- CometProject (7) + : : : : : +- CometFilter (6) + : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : : : +- ReusedExchange (13) + : : : +- BroadcastExchange (19) + : : : +- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.store (16) + : : +- BroadcastExchange (26) + : : +- * ColumnarToRow (25) + : : +- CometProject (24) + : : +- CometFilter (23) + : : +- CometScan parquet spark_catalog.default.item (22) + : +- BroadcastExchange (33) + : +- * ColumnarToRow (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.promotion (29) + :- * HashAggregate (72) + : +- * ColumnarToRow (71) + : +- CometColumnarExchange (70) + : +- RowToColumnar (69) + : +- * HashAggregate (68) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (61) + : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : :- * Project (55) + : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : :- * ColumnarToRow (52) + : : : : : +- CometProject (51) + : : : : : +- CometSortMergeJoin (50) + : : : : : :- CometSort (44) + : : : : : : +- CometColumnarExchange (43) + : : : : : : +- CometFilter (42) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) + : : : : : +- CometSort (49) + : : : : : +- CometColumnarExchange (48) + : : : : : +- CometProject (47) + : : : : : +- CometFilter (46) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) + : : : : +- ReusedExchange (53) + : : : +- BroadcastExchange (59) + : : : +- * ColumnarToRow (58) + : : : +- CometFilter (57) + : : : +- CometScan parquet spark_catalog.default.catalog_page (56) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + +- * HashAggregate (104) + +- * ColumnarToRow (103) + +- CometColumnarExchange (102) + +- RowToColumnar (101) + +- * HashAggregate (100) + +- * Project (99) + +- * BroadcastHashJoin Inner BuildRight (98) + :- * Project (96) + : +- * BroadcastHashJoin Inner BuildRight (95) + : :- * Project (93) + : : +- * BroadcastHashJoin Inner BuildRight (92) + : : :- * Project (87) + : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : :- * ColumnarToRow (84) + : : : : +- CometProject (83) + : : : : +- CometSortMergeJoin (82) + : : : : :- CometSort (76) + : : : : : +- CometColumnarExchange (75) + : : : : : +- CometFilter (74) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (73) + : : : : +- CometSort (81) + : : : : +- CometColumnarExchange (80) + : : : : +- CometProject (79) + : : : : +- CometFilter (78) + : : : : +- CometScan parquet spark_catalog.default.web_returns (77) + : : : +- ReusedExchange (85) + : : +- BroadcastExchange (91) + : : +- * ColumnarToRow (90) + : : +- CometFilter (89) + : : +- CometScan parquet spark_catalog.default.web_site (88) + : +- ReusedExchange (94) + +- ReusedExchange (97) (unknown) Scan parquet spark_catalog.default.store_sales @@ -117,9 +125,9 @@ ReadSchema: struct -(40) CometFilter +(42) CometFilter Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) -(41) CometExchange +(43) CometColumnarExchange Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(42) CometSort +(44) CometSort Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45], [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST] @@ -310,44 +324,44 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(44) CometFilter +(46) CometFilter Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) -(45) CometProject +(47) CometProject Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -(46) CometExchange +(48) CometColumnarExchange Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(47) CometSort +(49) CometSort Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST] -(48) CometSortMergeJoin +(50) CometSortMergeJoin Left output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Right output [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] Arguments: [cs_item_sk#40, cs_order_number#42], [cr_item_sk#47, cr_order_number#48], LeftOuter -(49) CometProject +(51) CometProject Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50], [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -(50) ColumnarToRow [codegen id : 11] +(52) ColumnarToRow [codegen id : 11] Input [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -(51) ReusedExchange [Reuses operator id: 109] +(53) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#52] -(52) BroadcastHashJoin [codegen id : 11] +(54) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#45] Right keys [1]: [d_date_sk#52] Join type: Inner Join condition: None -(53) Project [codegen id : 11] +(55) Project [codegen id : 11] Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] @@ -358,65 +372,71 @@ Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter +(57) CometFilter Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Condition : isnotnull(cp_catalog_page_sk#53) -(56) ColumnarToRow [codegen id : 8] +(58) ColumnarToRow [codegen id : 8] Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -(57) BroadcastExchange +(59) BroadcastExchange Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(58) BroadcastHashJoin [codegen id : 11] +(60) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_catalog_page_sk#39] Right keys [1]: [cp_catalog_page_sk#53] Join type: Inner Join condition: None -(59) Project [codegen id : 11] +(61) Project [codegen id : 11] Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] -(60) ReusedExchange [Reuses operator id: 26] +(62) ReusedExchange [Reuses operator id: 26] Output [1]: [i_item_sk#55] -(61) BroadcastHashJoin [codegen id : 11] +(63) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#40] Right keys [1]: [i_item_sk#55] Join type: Inner Join condition: None -(62) Project [codegen id : 11] +(64) Project [codegen id : 11] Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] -(63) ReusedExchange [Reuses operator id: 33] +(65) ReusedExchange [Reuses operator id: 33] Output [1]: [p_promo_sk#56] -(64) BroadcastHashJoin [codegen id : 11] +(66) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_promo_sk#41] Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(65) Project [codegen id : 11] +(67) Project [codegen id : 11] Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] -(66) HashAggregate [codegen id : 11] +(68) HashAggregate [codegen id : 11] Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -(67) Exchange +(69) RowToColumnar +Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] + +(70) CometColumnarExchange +Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(71) ColumnarToRow [codegen id : 12] Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(68) HashAggregate [codegen id : 12] +(72) HashAggregate [codegen id : 12] Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] @@ -431,15 +451,15 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(70) CometFilter +(74) CometFilter Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) -(71) CometExchange +(75) CometColumnarExchange Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(72) CometSort +(76) CometSort Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81], [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST] @@ -450,44 +470,44 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(74) CometFilter +(78) CometFilter Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) -(75) CometProject +(79) CometProject Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -(76) CometExchange +(80) CometColumnarExchange Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(77) CometSort +(81) CometSort Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST] -(78) CometSortMergeJoin +(82) CometSortMergeJoin Left output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] Right output [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] Arguments: [ws_item_sk#75, ws_order_number#78], [wr_item_sk#83, wr_order_number#84], LeftOuter -(79) CometProject +(83) CometProject Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86], [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -(80) ColumnarToRow [codegen id : 17] +(84) ColumnarToRow [codegen id : 17] Input [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -(81) ReusedExchange [Reuses operator id: 109] +(85) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#88] -(82) BroadcastHashJoin [codegen id : 17] +(86) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_sold_date_sk#81] Right keys [1]: [d_date_sk#88] Join type: Inner Join condition: None -(83) Project [codegen id : 17] +(87) Project [codegen id : 17] Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] @@ -498,133 +518,143 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter +(89) CometFilter Input [2]: [web_site_sk#89, web_site_id#90] Condition : isnotnull(web_site_sk#89) -(86) ColumnarToRow [codegen id : 14] +(90) ColumnarToRow [codegen id : 14] Input [2]: [web_site_sk#89, web_site_id#90] -(87) BroadcastExchange +(91) BroadcastExchange Input [2]: [web_site_sk#89, web_site_id#90] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(88) BroadcastHashJoin [codegen id : 17] +(92) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_web_site_sk#76] Right keys [1]: [web_site_sk#89] Join type: Inner Join condition: None -(89) Project [codegen id : 17] +(93) Project [codegen id : 17] Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] -(90) ReusedExchange [Reuses operator id: 26] +(94) ReusedExchange [Reuses operator id: 26] Output [1]: [i_item_sk#91] -(91) BroadcastHashJoin [codegen id : 17] +(95) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_item_sk#75] Right keys [1]: [i_item_sk#91] Join type: Inner Join condition: None -(92) Project [codegen id : 17] +(96) Project [codegen id : 17] Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] -(93) ReusedExchange [Reuses operator id: 33] +(97) ReusedExchange [Reuses operator id: 33] Output [1]: [p_promo_sk#92] -(94) BroadcastHashJoin [codegen id : 17] +(98) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_promo_sk#77] Right keys [1]: [p_promo_sk#92] Join type: Inner Join condition: None -(95) Project [codegen id : 17] +(99) Project [codegen id : 17] Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] -(96) HashAggregate [codegen id : 17] +(100) HashAggregate [codegen id : 17] Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Keys [1]: [web_site_id#90] Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -(97) Exchange +(101) RowToColumnar Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(98) HashAggregate [codegen id : 18] +(102) CometColumnarExchange +Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(103) ColumnarToRow [codegen id : 18] +Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] + +(104) HashAggregate [codegen id : 18] Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] Keys [1]: [web_site_id#90] Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#106, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#107, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#108, web channel AS channel#109, concat(web_site, web_site_id#90) AS id#110] -(99) Union +(105) Union -(100) Expand [codegen id : 19] +(106) Expand [codegen id : 19] Input [5]: [sales#34, returns#35, profit#36, channel#37, id#38] Arguments: [[sales#34, returns#35, profit#36, channel#37, id#38, 0], [sales#34, returns#35, profit#36, channel#37, null, 1], [sales#34, returns#35, profit#36, null, null, 3]], [sales#34, returns#35, profit#36, channel#111, id#112, spark_grouping_id#113] -(101) HashAggregate [codegen id : 19] +(107) HashAggregate [codegen id : 19] Input [6]: [sales#34, returns#35, profit#36, channel#111, id#112, spark_grouping_id#113] Keys [3]: [channel#111, id#112, spark_grouping_id#113] Functions [3]: [partial_sum(sales#34), partial_sum(returns#35), partial_sum(profit#36)] Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] Results [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -(102) Exchange +(108) RowToColumnar +Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(109) CometColumnarExchange Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Arguments: hashpartitioning(channel#111, id#112, spark_grouping_id#113, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: hashpartitioning(channel#111, id#112, spark_grouping_id#113, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(103) HashAggregate [codegen id : 20] +(110) CometHashAggregate Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] Keys [3]: [channel#111, id#112, spark_grouping_id#113] Functions [3]: [sum(sales#34), sum(returns#35), sum(profit#36)] -Aggregate Attributes [3]: [sum(sales#34)#126, sum(returns#35)#127, sum(profit#36)#128] -Results [5]: [channel#111, id#112, sum(sales#34)#126 AS sales#129, sum(returns#35)#127 AS returns#130, sum(profit#36)#128 AS profit#131] -(104) TakeOrderedAndProject -Input [5]: [channel#111, id#112, sales#129, returns#130, profit#131] -Arguments: 100, [channel#111 ASC NULLS FIRST, id#112 ASC NULLS FIRST], [channel#111, id#112, sales#129, returns#130, profit#131] +(111) CometTakeOrderedAndProject +Input [5]: [channel#111, id#112, sales#126, returns#127, profit#128] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#111 ASC NULLS FIRST,id#112 ASC NULLS FIRST], output=[channel#111,id#112,sales#126,returns#127,profit#128]), 100, [channel#111 ASC NULLS FIRST, id#112 ASC NULLS FIRST], [channel#111, id#112, sales#126, returns#127, profit#128] + +(112) ColumnarToRow [codegen id : 20] +Input [5]: [channel#111, id#112, sales#126, returns#127, profit#128] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (109) -+- * ColumnarToRow (108) - +- CometProject (107) - +- CometFilter (106) - +- CometScan parquet spark_catalog.default.date_dim (105) +BroadcastExchange (117) ++- * ColumnarToRow (116) + +- CometProject (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#132] +Output [2]: [d_date_sk#14, d_date#129] 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 -(106) CometFilter -Input [2]: [d_date_sk#14, d_date#132] -Condition : (((isnotnull(d_date#132) AND (d_date#132 >= 2000-08-23)) AND (d_date#132 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) +(114) CometFilter +Input [2]: [d_date_sk#14, d_date#129] +Condition : (((isnotnull(d_date#129) AND (d_date#129 >= 2000-08-23)) AND (d_date#129 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(107) CometProject -Input [2]: [d_date_sk#14, d_date#132] +(115) CometProject +Input [2]: [d_date_sk#14, d_date#129] Arguments: [d_date_sk#14], [d_date_sk#14] -(108) ColumnarToRow [codegen id : 1] +(116) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(109) BroadcastExchange +(117) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 69 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#81 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 8fc66cbe4d..f28c862efd 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,152 +1,160 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (20) - 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] - InputAdapter - Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (19) - 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 (6) - 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] - InputAdapter - Exchange [s_store_id] #2 - WholeStageCodegen (5) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - 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_ticket_number,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_ticket_number] - CometExchange [ss_item_sk,ss_ticket_number] #3 - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] - CometScan 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) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number] - 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] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] +WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [channel,id,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (19) + 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 (6) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + 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_ticket_number,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_ticket_number] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 + CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometScan 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) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number] + CometColumnarExchange [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] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter - ReusedExchange [d_date_sk] #4 + BroadcastExchange #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (12) - 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] - InputAdapter - Exchange [cp_catalog_page_id] #9 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - 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_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometExchange [cs_item_sk,cs_order_number] #10 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - CometScan 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] - CometExchange [cr_item_sk,cr_order_number] #11 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometProject [p_promo_sk] + CometFilter [p_channel_tv,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (12) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #9 + RowToColumnar + WholeStageCodegen (11) + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + 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_item_sk,cs_order_number,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number] + CometColumnarExchange [cs_item_sk,cs_order_number] #10 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometScan 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] + CometColumnarExchange [cr_item_sk,cr_order_number] #11 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [i_item_sk] #7 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (8) - ColumnarToRow + ReusedExchange [p_promo_sk] #8 + WholeStageCodegen (18) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #13 + RowToColumnar + WholeStageCodegen (17) + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + 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_order_number,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_order_number] + CometColumnarExchange [ws_item_sk,ws_order_number] #14 + CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometScan 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] + CometColumnarExchange [wr_item_sk,wr_order_number] #15 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - InputAdapter - ReusedExchange [i_item_sk] #7 - InputAdapter - ReusedExchange [p_promo_sk] #8 - WholeStageCodegen (18) - 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] - InputAdapter - Exchange [web_site_id] #13 - WholeStageCodegen (17) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - 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_order_number,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_order_number] - CometExchange [ws_item_sk,ws_order_number] #14 - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - CometScan 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] - CometExchange [wr_item_sk,wr_order_number] #15 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + BroadcastExchange #16 + WholeStageCodegen (14) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [i_item_sk] #7 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - InputAdapter - ReusedExchange [i_item_sk] #7 - InputAdapter - ReusedExchange [p_promo_sk] #8 + ReusedExchange [p_promo_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index d8dc396dd7..c31369315e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt @@ -1,52 +1,56 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (33) - : : +- * Filter (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * HashAggregate (28) - : : +- Exchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : +- BroadcastExchange (39) - : +- * ColumnarToRow (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.customer (36) - +- BroadcastExchange (45) - +- * ColumnarToRow (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.customer_address (42) +TakeOrderedAndProject (52) ++- * Project (51) + +- * BroadcastHashJoin Inner BuildRight (50) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometColumnarExchange (15) + : : : +- RowToColumnar (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan parquet spark_catalog.default.customer_address (7) + : : +- BroadcastExchange (37) + : : +- * ColumnarToRow (36) + : : +- CometFilter (35) + : : +- CometHashAggregate (34) + : : +- CometColumnarExchange (33) + : : +- CometHashAggregate (32) + : : +- CometHashAggregate (31) + : : +- CometColumnarExchange (30) + : : +- RowToColumnar (29) + : : +- * HashAggregate (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (24) + : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : :- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (19) + : : : +- ReusedExchange (22) + : : +- ReusedExchange (25) + : +- BroadcastExchange (43) + : +- * ColumnarToRow (42) + : +- CometFilter (41) + : +- CometScan parquet spark_catalog.default.customer (40) + +- BroadcastExchange (49) + +- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.customer_address (46) (unknown) Scan parquet spark_catalog.default.catalog_returns @@ -64,7 +68,7 @@ Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_custom (3) ColumnarToRow [codegen id : 3] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -(4) ReusedExchange [Reuses operator id: 53] +(4) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -112,208 +116,212 @@ Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum#9] Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] -(14) Exchange +(14) RowToColumnar Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 11] +(15) CometColumnarExchange +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] Keys [2]: [cr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#11] -Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] -(16) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) +(17) CometFilter +Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +Condition : isnotnull(ctr_total_return#13) + +(18) ColumnarToRow [codegen id : 10] +Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] (unknown) Scan parquet spark_catalog.default.catalog_returns Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#14)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(18) CometFilter +(20) CometFilter Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Condition : isnotnull(cr_returning_addr_sk#2) -(19) ColumnarToRow [codegen id : 6] +(21) ColumnarToRow [codegen id : 6] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -(20) ReusedExchange [Reuses operator id: 53] +(22) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#6] -(21) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] -(23) ReusedExchange [Reuses operator id: 10] +(25) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#7, ca_state#8] -(24) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(27) Project [codegen id : 6] Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#8] -(26) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 6] Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] Keys [2]: [cr_returning_customer_sk#1, ca_state#8] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#16] -Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#16] -(27) Exchange -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(29) RowToColumnar +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#16] -(28) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] +(30) CometColumnarExchange +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#16] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(31) CometHashAggregate +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#16] Keys [2]: [cr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#11] -Results [2]: [ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] - -(29) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#13, ctr_total_return#14] -Keys [1]: [ctr_state#13] -Functions [1]: [partial_avg(ctr_total_return#14)] -Aggregate Attributes [2]: [sum#18, count#19] -Results [3]: [ctr_state#13, sum#20, count#21] - -(30) Exchange -Input [3]: [ctr_state#13, sum#20, count#21] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#13, sum#20, count#21] -Keys [1]: [ctr_state#13] -Functions [1]: [avg(ctr_total_return#14)] -Aggregate Attributes [1]: [avg(ctr_total_return#14)#22] -Results [2]: [(avg(ctr_total_return#14)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#13 AS ctr_state#13#24] - -(32) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) - -(33) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] + +(32) CometHashAggregate +Input [2]: [ctr_state#12, ctr_total_return#13] +Keys [1]: [ctr_state#12] +Functions [1]: [partial_avg(ctr_total_return#13)] + +(33) CometColumnarExchange +Input [3]: [ctr_state#12, sum#17, count#18] +Arguments: hashpartitioning(ctr_state#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometHashAggregate +Input [3]: [ctr_state#12, sum#17, count#18] +Keys [1]: [ctr_state#12] +Functions [1]: [avg(ctr_total_return#13)] + +(35) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#19) + +(36) ColumnarToRow [codegen id : 7] +Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] + +(37) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#13] -Right keys [1]: [ctr_state#13#24] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ctr_state#12] +Right keys [1]: [ctr_state#12#20] Join type: Inner -Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) +Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#19) -(35) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] +(39) Project [codegen id : 10] +Output [2]: [ctr_customer_sk#11, ctr_total_return#13] +Input [5]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] (unknown) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] +Output [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(37) CometFilter -Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) +(41) CometFilter +Input [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#23)) -(38) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] +(42) ColumnarToRow [codegen id : 8] +Input [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] -(39) BroadcastExchange -Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] +(43) BroadcastExchange +Input [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#12] -Right keys [1]: [c_customer_sk#25] +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ctr_customer_sk#11] +Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(41) Project [codegen id : 11] -Output [6]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] -Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] +(45) Project [codegen id : 10] +Output [6]: [ctr_total_return#13, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] +Input [8]: [ctr_customer_sk#11, ctr_total_return#13, c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] (unknown) Scan parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] +Output [12]: [ca_address_sk#27, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(43) CometFilter -Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -Condition : ((isnotnull(ca_state#38) AND (ca_state#38 = GA)) AND isnotnull(ca_address_sk#31)) +(47) CometFilter +Input [12]: [ca_address_sk#27, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38] +Condition : ((isnotnull(ca_state#34) AND (ca_state#34 = GA)) AND isnotnull(ca_address_sk#27)) -(44) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] +(48) ColumnarToRow [codegen id : 9] +Input [12]: [ca_address_sk#27, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38] -(45) BroadcastExchange -Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] +(49) BroadcastExchange +Input [12]: [ca_address_sk#27, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#27] -Right keys [1]: [ca_address_sk#31] +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_current_addr_sk#23] +Right keys [1]: [ca_address_sk#27] Join type: Inner Join condition: None -(47) Project [codegen id : 11] -Output [16]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] -Input [18]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] +(51) Project [codegen id : 10] +Output [16]: [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38, ctr_total_return#13] +Input [18]: [ctr_total_return#13, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, ca_address_sk#27, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38] -(48) TakeOrderedAndProject -Input [16]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] -Arguments: 100, [c_customer_id#26 ASC NULLS FIRST, c_salutation#28 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, c_last_name#30 ASC NULLS FIRST, ca_street_number#32 ASC NULLS FIRST, ca_street_name#33 ASC NULLS FIRST, ca_street_type#34 ASC NULLS FIRST, ca_suite_number#35 ASC NULLS FIRST, ca_city#36 ASC NULLS FIRST, ca_county#37 ASC NULLS FIRST, ca_state#38 ASC NULLS FIRST, ca_zip#39 ASC NULLS FIRST, ca_country#40 ASC NULLS FIRST, ca_gmt_offset#41 ASC NULLS FIRST, ca_location_type#42 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] +(52) TakeOrderedAndProject +Input [16]: [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38, ctr_total_return#13] +Arguments: 100, [c_customer_id#22 ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_first_name#25 ASC NULLS FIRST, c_last_name#26 ASC NULLS FIRST, ca_street_number#28 ASC NULLS FIRST, ca_street_name#29 ASC NULLS FIRST, ca_street_type#30 ASC NULLS FIRST, ca_suite_number#31 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, ca_state#34 ASC NULLS FIRST, ca_zip#35 ASC NULLS FIRST, ca_country#36 ASC NULLS FIRST, ca_gmt_offset#37 ASC NULLS FIRST, ca_location_type#38 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38, ctr_total_return#13] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#43] +Output [2]: [d_date_sk#6, d_year#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter -Input [2]: [d_date_sk#6, d_year#43] -Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#6)) +(54) CometFilter +Input [2]: [d_date_sk#6, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2000)) AND isnotnull(d_date_sk#6)) -(51) CometProject -Input [2]: [d_date_sk#6, d_year#43] +(55) CometProject +Input [2]: [d_date_sk#6, d_year#39] Arguments: [d_date_sk#6], [d_date_sk#6] -(52) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(53) BroadcastExchange +(57) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 17 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index da163f0233..8e5f0eb89b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -1,54 +1,56 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - WholeStageCodegen (11) + WholeStageCodegen (10) Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk,ctr_total_return] BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow + ColumnarToRow + InputAdapter + CometFilter [ctr_total_return] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum] + CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 - WholeStageCodegen (8) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] - InputAdapter - Exchange [ctr_state] #5 - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [(avg(ctr_total_return) * 1.2)] + CometHashAggregate [ctr_state,sum,count] + CometColumnarExchange [ctr_state] #5 + CometHashAggregate [ctr_state,ctr_total_return] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum] + CometColumnarExchange [cr_returning_customer_sk,ca_state] #6 + RowToColumnar WholeStageCodegen (6) HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] @@ -66,14 +68,14 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 - WholeStageCodegen (9) + WholeStageCodegen (8) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 - WholeStageCodegen (10) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometFilter [ca_state,ca_address_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt index 683f7bd940..694fc26e95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt @@ -1,29 +1,31 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * ColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan parquet spark_catalog.default.store_sales (16) +* ColumnarToRow (27) ++- CometTakeOrderedAndProject (26) + +- CometHashAggregate (25) + +- CometColumnarExchange (24) + +- RowToColumnar (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildLeft (20) + :- BroadcastExchange (15) + : +- * Project (14) + : +- * BroadcastHashJoin Inner BuildRight (13) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (9) + : : +- * ColumnarToRow (8) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.inventory (5) + : +- ReusedExchange (12) + +- * ColumnarToRow (19) + +- CometProject (18) + +- CometFilter (17) + +- CometScan parquet spark_catalog.default.store_sales (16) (unknown) Scan parquet spark_catalog.default.item @@ -77,7 +79,7 @@ Join condition: None Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_item_sk#6, inv_date_sk#8] -(12) ReusedExchange [Reuses operator id: 30] +(12) ReusedExchange [Reuses operator id: 32] Output [1]: [d_date_sk#10] (13) BroadcastHashJoin [codegen id : 3] @@ -129,29 +131,33 @@ Functions: [] Aggregate Attributes: [] Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(23) Exchange +(23) RowToColumnar Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) HashAggregate [codegen id : 5] +(24) CometColumnarExchange +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(25) CometHashAggregate Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(25) TakeOrderedAndProject +(26) CometTakeOrderedAndProject +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] + +(27) ColumnarToRow [codegen id : 5] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (30) -+- * ColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.date_dim (26) +BroadcastExchange (32) ++- * ColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan parquet spark_catalog.default.date_dim (28) (unknown) Scan parquet spark_catalog.default.date_dim @@ -161,18 +167,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter +(29) CometFilter Input [2]: [d_date_sk#10, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-05-25)) AND (d_date#13 <= 2000-07-24)) AND isnotnull(d_date_sk#10)) -(28) CometProject +(30) CometProject Input [2]: [d_date_sk#10, d_date#13] Arguments: [d_date_sk#10], [d_date_sk#10] -(29) ColumnarToRow [codegen id : 1] +(31) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(30) BroadcastExchange +(32) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt index 0252eb575f..2c652d64fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt @@ -1,44 +1,46 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - WholeStageCodegen (5) - HashAggregate [i_item_id,i_item_desc,i_current_price] - InputAdapter - Exchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] +WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,ss_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (3) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - ColumnarToRow - InputAdapter - CometProject [ss_item_sk] - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + ReusedExchange [d_date_sk] #4 + ColumnarToRow + InputAdapter + CometProject [ss_item_sk] + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 164ecf8af8..63c12d9f08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,50 +1,56 @@ == Physical Plan == -TakeOrderedAndProject (46) -+- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_returns (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (10) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- BroadcastExchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometFilter (32) - : : +- CometScan parquet spark_catalog.default.web_returns (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) +TakeOrderedAndProject (52) ++- * Project (51) + +- * BroadcastHashJoin Inner BuildRight (50) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * ColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometColumnarExchange (15) + : : +- RowToColumnar (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_returns (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- ReusedExchange (10) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometHashAggregate (30) + : +- CometColumnarExchange (29) + : +- RowToColumnar (28) + : +- * HashAggregate (27) + : +- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * ColumnarToRow (20) + : : : +- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) + : : +- ReusedExchange (21) + : +- ReusedExchange (24) + +- BroadcastExchange (49) + +- * ColumnarToRow (48) + +- CometHashAggregate (47) + +- CometColumnarExchange (46) + +- RowToColumnar (45) + +- * HashAggregate (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * ColumnarToRow (37) + : : +- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.web_returns (35) + : +- ReusedExchange (38) + +- ReusedExchange (41) (unknown) Scan parquet spark_catalog.default.store_returns @@ -90,7 +96,7 @@ Join condition: None Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#6] -(10) ReusedExchange [Reuses operator id: 62] +(10) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#7] (11) BroadcastHashJoin [codegen id : 5] @@ -110,262 +116,274 @@ Functions [1]: [partial_sum(sr_return_quantity#2)] Aggregate Attributes [1]: [sum#8] Results [2]: [i_item_id#6, sum#9] -(14) Exchange +(14) RowToColumnar Input [2]: [i_item_id#6, sum#9] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 18] +(15) CometColumnarExchange +Input [2]: [i_item_id#6, sum#9] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [2]: [i_item_id#6, sum#9] Keys [1]: [i_item_id#6] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#10] -Results [2]: [i_item_id#6 AS item_id#11, sum(sr_return_quantity#2)#10 AS sr_item_qty#12] + +(17) ColumnarToRow [codegen id : 18] +Input [2]: [item_id#10, sr_item_qty#11] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] +Output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#15), dynamicpruningexpression(cr_returned_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cr_returned_date_sk#14), dynamicpruningexpression(cr_returned_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(17) CometFilter -Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] -Condition : isnotnull(cr_item_sk#13) +(19) CometFilter +Input [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] +Condition : isnotnull(cr_item_sk#12) -(18) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] +(20) ColumnarToRow [codegen id : 10] +Input [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -(19) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#17, i_item_id#18] +(21) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#16, i_item_id#17] -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#13] -Right keys [1]: [i_item_sk#17] +(22) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#12] +Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(21) Project [codegen id : 10] -Output [3]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18] -Input [5]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15, i_item_sk#17, i_item_id#18] +(23) Project [codegen id : 10] +Output [3]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] +Input [5]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14, i_item_sk#16, i_item_id#17] -(22) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#19] +(24) ReusedExchange [Reuses operator id: 68] +Output [1]: [d_date_sk#18] -(23) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#15] -Right keys [1]: [d_date_sk#19] +(25) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#14] +Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(24) Project [codegen id : 10] -Output [2]: [cr_return_quantity#14, i_item_id#18] -Input [4]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18, d_date_sk#19] - -(25) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#14, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(cr_return_quantity#14)] -Aggregate Attributes [1]: [sum#20] -Results [2]: [i_item_id#18, sum#21] - -(26) Exchange -Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#18, sum#21] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(cr_return_quantity#14)] -Aggregate Attributes [1]: [sum(cr_return_quantity#14)#22] -Results [2]: [i_item_id#18 AS item_id#23, sum(cr_return_quantity#14)#22 AS cr_item_qty#24] - -(28) BroadcastExchange -Input [2]: [item_id#23, cr_item_qty#24] +(26) Project [codegen id : 10] +Output [2]: [cr_return_quantity#13, i_item_id#17] +Input [4]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17, d_date_sk#18] + +(27) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#13, i_item_id#17] +Keys [1]: [i_item_id#17] +Functions [1]: [partial_sum(cr_return_quantity#13)] +Aggregate Attributes [1]: [sum#19] +Results [2]: [i_item_id#17, sum#20] + +(28) RowToColumnar +Input [2]: [i_item_id#17, sum#20] + +(29) CometColumnarExchange +Input [2]: [i_item_id#17, sum#20] +Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(30) CometHashAggregate +Input [2]: [i_item_id#17, sum#20] +Keys [1]: [i_item_id#17] +Functions [1]: [sum(cr_return_quantity#13)] + +(31) ColumnarToRow [codegen id : 11] +Input [2]: [item_id#21, cr_item_qty#22] + +(32) BroadcastExchange +Input [2]: [item_id#21, cr_item_qty#22] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#11] -Right keys [1]: [item_id#23] +(33) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#10] +Right keys [1]: [item_id#21] Join type: Inner Join condition: None -(30) Project [codegen id : 18] -Output [3]: [item_id#11, sr_item_qty#12, cr_item_qty#24] -Input [4]: [item_id#11, sr_item_qty#12, item_id#23, cr_item_qty#24] +(34) Project [codegen id : 18] +Output [3]: [item_id#10, sr_item_qty#11, cr_item_qty#22] +Input [4]: [item_id#10, sr_item_qty#11, item_id#21, cr_item_qty#22] (unknown) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] +Output [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(wr_returned_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(wr_returned_date_sk#25), dynamicpruningexpression(wr_returned_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Condition : isnotnull(wr_item_sk#25) +(36) CometFilter +Input [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] +Condition : isnotnull(wr_item_sk#23) -(33) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] +(37) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] -(34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#29, i_item_id#30] +(38) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#27, i_item_id#28] -(35) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#25] -Right keys [1]: [i_item_sk#29] +(39) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#23] +Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(36) Project [codegen id : 16] -Output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] -Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#29, i_item_id#30] +(40) Project [codegen id : 16] +Output [3]: [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28] +Input [5]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25, i_item_sk#27, i_item_id#28] -(37) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#31] +(41) ReusedExchange [Reuses operator id: 68] +Output [1]: [d_date_sk#29] -(38) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#27] -Right keys [1]: [d_date_sk#31] +(42) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#25] +Right keys [1]: [d_date_sk#29] Join type: Inner Join condition: None -(39) Project [codegen id : 16] -Output [2]: [wr_return_quantity#26, i_item_id#30] -Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30, d_date_sk#31] - -(40) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#26, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(wr_return_quantity#26)] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#30, sum#33] - -(41) Exchange -Input [2]: [i_item_id#30, sum#33] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(42) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#30, sum#33] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(wr_return_quantity#26)] -Aggregate Attributes [1]: [sum(wr_return_quantity#26)#34] -Results [2]: [i_item_id#30 AS item_id#35, sum(wr_return_quantity#26)#34 AS wr_item_qty#36] - -(43) BroadcastExchange -Input [2]: [item_id#35, wr_item_qty#36] +(43) Project [codegen id : 16] +Output [2]: [wr_return_quantity#24, i_item_id#28] +Input [4]: [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28, d_date_sk#29] + +(44) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#24, i_item_id#28] +Keys [1]: [i_item_id#28] +Functions [1]: [partial_sum(wr_return_quantity#24)] +Aggregate Attributes [1]: [sum#30] +Results [2]: [i_item_id#28, sum#31] + +(45) RowToColumnar +Input [2]: [i_item_id#28, sum#31] + +(46) CometColumnarExchange +Input [2]: [i_item_id#28, sum#31] +Arguments: hashpartitioning(i_item_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(47) CometHashAggregate +Input [2]: [i_item_id#28, sum#31] +Keys [1]: [i_item_id#28] +Functions [1]: [sum(wr_return_quantity#24)] + +(48) ColumnarToRow [codegen id : 17] +Input [2]: [item_id#32, wr_item_qty#33] + +(49) BroadcastExchange +Input [2]: [item_id#32, wr_item_qty#33] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#11] -Right keys [1]: [item_id#35] +(50) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#10] +Right keys [1]: [item_id#32] Join type: Inner Join condition: None -(45) Project [codegen id : 18] -Output [8]: [item_id#11, sr_item_qty#12, (((cast(sr_item_qty#12 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS sr_dev#37, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS cr_dev#38, wr_item_qty#36, (((cast(wr_item_qty#36 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS wr_dev#39, (cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as decimal(20,0)) / 3.0) AS average#40] -Input [5]: [item_id#11, sr_item_qty#12, cr_item_qty#24, item_id#35, wr_item_qty#36] +(51) Project [codegen id : 18] +Output [8]: [item_id#10, sr_item_qty#11, (((cast(sr_item_qty#11 as double) / cast(((sr_item_qty#11 + cr_item_qty#22) + wr_item_qty#33) as double)) / 3.0) * 100.0) AS sr_dev#34, cr_item_qty#22, (((cast(cr_item_qty#22 as double) / cast(((sr_item_qty#11 + cr_item_qty#22) + wr_item_qty#33) as double)) / 3.0) * 100.0) AS cr_dev#35, wr_item_qty#33, (((cast(wr_item_qty#33 as double) / cast(((sr_item_qty#11 + cr_item_qty#22) + wr_item_qty#33) as double)) / 3.0) * 100.0) AS wr_dev#36, (cast(((sr_item_qty#11 + cr_item_qty#22) + wr_item_qty#33) as decimal(20,0)) / 3.0) AS average#37] +Input [5]: [item_id#10, sr_item_qty#11, cr_item_qty#22, item_id#32, wr_item_qty#33] -(46) TakeOrderedAndProject -Input [8]: [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] -Arguments: 100, [item_id#11 ASC NULLS FIRST, sr_item_qty#12 ASC NULLS FIRST], [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] +(52) TakeOrderedAndProject +Input [8]: [item_id#10, sr_item_qty#11, sr_dev#34, cr_item_qty#22, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] +Arguments: 100, [item_id#10 ASC NULLS FIRST, sr_item_qty#11 ASC NULLS FIRST], [item_id#10, sr_item_qty#11, sr_dev#34, cr_item_qty#22, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * Project (61) - +- * BroadcastHashJoin LeftSemi BuildRight (60) - :- * ColumnarToRow (49) - : +- CometFilter (48) - : +- CometScan parquet spark_catalog.default.date_dim (47) - +- BroadcastExchange (59) - +- * Project (58) - +- * BroadcastHashJoin LeftSemi BuildRight (57) - :- * ColumnarToRow (51) - : +- CometScan parquet spark_catalog.default.date_dim (50) - +- BroadcastExchange (56) - +- * ColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan parquet spark_catalog.default.date_dim (52) +BroadcastExchange (68) ++- * Project (67) + +- * BroadcastHashJoin LeftSemi BuildRight (66) + :- * ColumnarToRow (55) + : +- CometFilter (54) + : +- CometScan parquet spark_catalog.default.date_dim (53) + +- BroadcastExchange (65) + +- * Project (64) + +- * BroadcastHashJoin LeftSemi BuildRight (63) + :- * ColumnarToRow (57) + : +- CometScan parquet spark_catalog.default.date_dim (56) + +- BroadcastExchange (62) + +- * ColumnarToRow (61) + +- CometProject (60) + +- CometFilter (59) + +- CometScan parquet spark_catalog.default.date_dim (58) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_date#41] +Output [2]: [d_date_sk#7, d_date#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter -Input [2]: [d_date_sk#7, d_date#41] +(54) CometFilter +Input [2]: [d_date_sk#7, d_date#38] Condition : isnotnull(d_date_sk#7) -(49) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#7, d_date#41] +(55) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#7, d_date#38] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#42, d_week_seq#43] +Output [2]: [d_date#39, d_week_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(51) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#42, d_week_seq#43] +(57) ColumnarToRow [codegen id : 2] +Input [2]: [d_date#39, d_week_seq#40] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#44, d_week_seq#45] +Output [2]: [d_date#41, d_week_seq#42] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(53) CometFilter -Input [2]: [d_date#44, d_week_seq#45] -Condition : cast(d_date#44 as string) IN (2000-06-30,2000-09-27,2000-11-17) +(59) CometFilter +Input [2]: [d_date#41, d_week_seq#42] +Condition : cast(d_date#41 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(54) CometProject -Input [2]: [d_date#44, d_week_seq#45] -Arguments: [d_week_seq#45], [d_week_seq#45] +(60) CometProject +Input [2]: [d_date#41, d_week_seq#42] +Arguments: [d_week_seq#42], [d_week_seq#42] -(55) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#45] +(61) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#42] -(56) BroadcastExchange -Input [1]: [d_week_seq#45] +(62) BroadcastExchange +Input [1]: [d_week_seq#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#43] -Right keys [1]: [d_week_seq#45] +(63) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [d_week_seq#40] +Right keys [1]: [d_week_seq#42] Join type: LeftSemi Join condition: None -(58) Project [codegen id : 2] -Output [1]: [d_date#42] -Input [2]: [d_date#42, d_week_seq#43] +(64) Project [codegen id : 2] +Output [1]: [d_date#39] +Input [2]: [d_date#39, d_week_seq#40] -(59) BroadcastExchange -Input [1]: [d_date#42] +(65) BroadcastExchange +Input [1]: [d_date#39] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#41] -Right keys [1]: [d_date#42] +(66) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date#38] +Right keys [1]: [d_date#39] Join type: LeftSemi Join condition: None -(61) Project [codegen id : 3] +(67) Project [codegen id : 3] Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#41] +Input [2]: [d_date_sk#7, d_date#38] -(62) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 16 Hosting Expression = cr_returned_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 18 Hosting Expression = cr_returned_date_sk#14 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 31 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 35 Hosting Expression = wr_returned_date_sk#25 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index a8f1ba3f10..bc1df2c571 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -4,92 +4,98 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty BroadcastHashJoin [item_id,item_id] Project [item_id,sr_item_qty,cr_item_qty] BroadcastHashJoin [item_id,item_id] - HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] + ColumnarToRow InputAdapter - Exchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - Project [sr_return_quantity,i_item_id] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [sr_return_quantity,sr_returned_date_sk,i_item_id] - BroadcastHashJoin [sr_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (3) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - ColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_item_id,sr_return_quantity] [sum,sum] + Project [sr_return_quantity,i_item_id] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [sr_return_quantity,sr_returned_date_sk,i_item_id] + BroadcastHashJoin [sr_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (3) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + ColumnarToRow + InputAdapter + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [d_date] + BroadcastHashJoin [d_week_seq,d_week_seq] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_date] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (11) - HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] + ColumnarToRow InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (10) - HashAggregate [i_item_id,cr_return_quantity] [sum,sum] - Project [cr_return_quantity,i_item_id] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cr_return_quantity,cr_returned_date_sk,i_item_id] - BroadcastHashJoin [cr_item_sk,i_item_sk] + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #7 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [i_item_id,cr_return_quantity] [sum,sum] + Project [cr_return_quantity,i_item_id] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Project [cr_return_quantity,cr_returned_date_sk,i_item_id] + BroadcastHashJoin [cr_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (17) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,sum] + CometColumnarExchange [i_item_id] #9 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [i_item_id,wr_return_quantity] [sum,sum] + Project [wr_return_quantity,i_item_id] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Project [wr_return_quantity,wr_returned_date_sk,i_item_id] + BroadcastHashJoin [wr_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometFilter [cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + CometFilter [wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 InputAdapter ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] - InputAdapter - Exchange [i_item_id] #9 - WholeStageCodegen (16) - HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - Project [wr_return_quantity,i_item_id] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Project [wr_return_quantity,wr_returned_date_sk,i_item_id] - BroadcastHashJoin [wr_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index 53329906e1..e5c29723a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -1,51 +1,53 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Project (27) - : : : +- * BroadcastHashJoin Inner BuildRight (26) - : : : :- * Project (21) - : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometBroadcastExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.web_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.web_page (10) - : : : : +- BroadcastExchange (19) - : : : : +- * ColumnarToRow (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (16) - : : : +- BroadcastExchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : +- BroadcastExchange (32) - : : +- * ColumnarToRow (31) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometScan parquet spark_catalog.default.customer_address (28) - : +- ReusedExchange (35) - +- BroadcastExchange (41) - +- * ColumnarToRow (40) - +- CometFilter (39) - +- CometScan parquet spark_catalog.default.reason (38) +* ColumnarToRow (49) ++- CometTakeOrderedAndProject (48) + +- CometHashAggregate (47) + +- CometColumnarExchange (46) + +- RowToColumnar (45) + +- * HashAggregate (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (34) + : : +- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Project (27) + : : : +- * BroadcastHashJoin Inner BuildRight (26) + : : : :- * Project (21) + : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometBroadcastExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.web_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.web_page (10) + : : : : +- BroadcastExchange (19) + : : : : +- * ColumnarToRow (18) + : : : : +- CometFilter (17) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (16) + : : : +- BroadcastExchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) + : : +- BroadcastExchange (32) + : : +- * ColumnarToRow (31) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.customer_address (28) + : +- ReusedExchange (35) + +- BroadcastExchange (41) + +- * ColumnarToRow (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.reason (38) (unknown) Scan parquet spark_catalog.default.web_sales @@ -207,7 +209,7 @@ Join condition: ((((ca_state#26 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) A Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#25, ca_state#26] -(35) ReusedExchange [Reuses operator id: 52] +(35) ReusedExchange [Reuses operator id: 54] Output [1]: [d_date_sk#28] (36) BroadcastHashJoin [codegen id : 7] @@ -255,50 +257,54 @@ Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunde Aggregate Attributes [6]: [sum#31, count#32, sum#33, count#34, sum#35, count#36] Results [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] -(45) Exchange +(45) RowToColumnar Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Arguments: hashpartitioning(r_reason_desc#30, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(46) HashAggregate [codegen id : 8] +(46) CometColumnarExchange +Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] +Arguments: hashpartitioning(r_reason_desc#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(47) CometHashAggregate Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] Keys [1]: [r_reason_desc#30] Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] -Aggregate Attributes [3]: [avg(ws_quantity#4)#43, avg(UnscaledValue(wr_refunded_cash#16))#44, avg(UnscaledValue(wr_fee#15))#45] -Results [4]: [substr(r_reason_desc#30, 1, 20) AS substr(r_reason_desc, 1, 20)#46, avg(ws_quantity#4)#43 AS avg(ws_quantity)#47, cast((avg(UnscaledValue(wr_refunded_cash#16))#44 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#48, cast((avg(UnscaledValue(wr_fee#15))#45 / 100.0) as decimal(11,6)) AS avg(wr_fee)#49] -(47) TakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] -Arguments: 100, [substr(r_reason_desc, 1, 20)#46 ASC NULLS FIRST, avg(ws_quantity)#47 ASC NULLS FIRST, avg(wr_refunded_cash)#48 ASC NULLS FIRST, avg(wr_fee)#49 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] +(48) CometTakeOrderedAndProject +Input [4]: [substr(r_reason_desc, 1, 20)#43, avg(ws_quantity)#44, avg(wr_refunded_cash)#45, avg(wr_fee)#46] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#43 ASC NULLS FIRST,avg(ws_quantity)#44 ASC NULLS FIRST,avg(wr_refunded_cash)#45 ASC NULLS FIRST,avg(wr_fee)#46 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#43,avg(ws_quantity)#44,avg(wr_refunded_cash)#45,avg(wr_fee)#46]), 100, [substr(r_reason_desc, 1, 20)#43 ASC NULLS FIRST, avg(ws_quantity)#44 ASC NULLS FIRST, avg(wr_refunded_cash)#45 ASC NULLS FIRST, avg(wr_fee)#46 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#43, avg(ws_quantity)#44, avg(wr_refunded_cash)#45, avg(wr_fee)#46] + +(49) ColumnarToRow [codegen id : 8] +Input [4]: [substr(r_reason_desc, 1, 20)#43, avg(ws_quantity)#44, avg(wr_refunded_cash)#45, avg(wr_fee)#46] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#50] +Output [2]: [d_date_sk#28, d_year#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter -Input [2]: [d_date_sk#28, d_year#50] -Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2000)) AND isnotnull(d_date_sk#28)) +(51) CometFilter +Input [2]: [d_date_sk#28, d_year#47] +Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2000)) AND isnotnull(d_date_sk#28)) -(50) CometProject -Input [2]: [d_date_sk#28, d_year#50] +(52) CometProject +Input [2]: [d_date_sk#28, d_year#47] Arguments: [d_date_sk#28], [d_date_sk#28] -(51) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] -(52) BroadcastExchange +(54) BroadcastExchange Input [1]: [d_date_sk#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt index 7c5ee727b7..4d1bf754c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt @@ -1,75 +1,77 @@ -TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (8) - HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] - InputAdapter - Exchange [r_reason_desc] #1 - WholeStageCodegen (7) - HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - BroadcastHashJoin [wr_reason_sk,r_reason_sk] - Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] - Project [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - ColumnarToRow - InputAdapter - CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - CometBroadcastExchange #2 - CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] +WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] + CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count] + CometColumnarExchange [r_reason_desc] #1 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] + BroadcastHashJoin [wr_reason_sk,r_reason_sk] + Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] + BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + Project [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + ColumnarToRow + InputAdapter + CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + CometBroadcastExchange #2 + CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + BroadcastExchange #6 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) + BroadcastExchange #7 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + BroadcastExchange #8 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [r_reason_sk] + CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt index cab784da58..5bd7bc5594 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt @@ -1,25 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * Project (20) - +- Window (19) - +- * Sort (18) - +- Exchange (17) - +- * HashAggregate (16) - +- Exchange (15) - +- * HashAggregate (14) - +- * Expand (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (10) - +- * ColumnarToRow (9) - +- CometFilter (8) - +- CometScan parquet spark_catalog.default.item (7) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * ColumnarToRow (20) + +- CometSort (19) + +- CometColumnarExchange (18) + +- CometHashAggregate (17) + +- CometColumnarExchange (16) + +- RowToColumnar (15) + +- * HashAggregate (14) + +- * Expand (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- ReusedExchange (4) + +- BroadcastExchange (10) + +- * ColumnarToRow (9) + +- CometFilter (8) + +- CometScan parquet spark_catalog.default.item (7) (unknown) Scan parquet spark_catalog.default.web_sales @@ -37,7 +39,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 26] +(4) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 3] @@ -89,66 +91,70 @@ Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum#12] Results [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] -(15) Exchange +(15) RowToColumnar Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] -Arguments: hashpartitioning(i_category#9, i_class#10, spark_grouping_id#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(16) HashAggregate [codegen id : 4] +(16) CometColumnarExchange +Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] +Arguments: hashpartitioning(i_category#9, i_class#10, spark_grouping_id#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(17) CometHashAggregate Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] Keys [3]: [i_category#9, i_class#10, spark_grouping_id#11] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#14] -Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS total_sum#15, i_category#9, i_class#10, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS lochierarchy#16, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS _w0#17, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS _w1#18, CASE WHEN (cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint) = 0) THEN i_category#9 END AS _w2#19] -(17) Exchange -Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] -Arguments: hashpartitioning(_w1#18, _w2#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(18) CometColumnarExchange +Input [7]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18] +Arguments: hashpartitioning(_w1#17, _w2#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(19) CometSort +Input [7]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18] +Arguments: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18], [_w1#17 ASC NULLS FIRST, _w2#18 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] -(18) Sort [codegen id : 5] -Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] -Arguments: [_w1#18 ASC NULLS FIRST, _w2#19 ASC NULLS FIRST, _w0#17 DESC NULLS LAST], false, 0 +(20) ColumnarToRow [codegen id : 4] +Input [7]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18] -(19) Window -Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] -Arguments: [rank(_w0#17) windowspecdefinition(_w1#18, _w2#19, _w0#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#20], [_w1#18, _w2#19], [_w0#17 DESC NULLS LAST] +(21) Window +Input [7]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18] +Arguments: [rank(_w0#16) windowspecdefinition(_w1#17, _w2#18, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#19], [_w1#17, _w2#18], [_w0#16 DESC NULLS LAST] -(20) Project [codegen id : 6] -Output [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] -Input [8]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19, rank_within_parent#20] +(22) Project [codegen id : 5] +Output [5]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, rank_within_parent#19] +Input [8]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18, rank_within_parent#19] -(21) TakeOrderedAndProject -Input [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] -Arguments: 100, [lochierarchy#16 DESC NULLS LAST, CASE WHEN (lochierarchy#16 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#20 ASC NULLS FIRST], [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] +(23) TakeOrderedAndProject +Input [5]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, rank_within_parent#19] +Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#19 ASC NULLS FIRST], [total_sum#14, i_category#9, i_class#10, lochierarchy#15, rank_within_parent#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (26) -+- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#21] +Output [2]: [d_date_sk#5, d_month_seq#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(23) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#21] -Condition : (((isnotnull(d_month_seq#21) AND (d_month_seq#21 >= 1200)) AND (d_month_seq#21 <= 1211)) AND isnotnull(d_date_sk#5)) +(25) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#20] +Condition : (((isnotnull(d_month_seq#20) AND (d_month_seq#20 >= 1200)) AND (d_month_seq#20 <= 1211)) AND isnotnull(d_date_sk#5)) -(24) CometProject -Input [2]: [d_date_sk#5, d_month_seq#21] +(26) CometProject +Input [2]: [d_date_sk#5, d_month_seq#20] Arguments: [d_date_sk#5], [d_date_sk#5] -(25) ColumnarToRow [codegen id : 1] +(27) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(26) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index f9db2ce7a4..b4fcc80345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -1,16 +1,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (6) + WholeStageCodegen (5) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (5) - Sort [_w1,_w2,_w0] + WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - InputAdapter - Exchange [i_category,i_class,spark_grouping_id] #2 + CometSort [_w1,_w2,_w0] + CometColumnarExchange [_w1,_w2] #1 + CometHashAggregate [i_category,i_class,spark_grouping_id,sum] + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + RowToColumnar WholeStageCodegen (3) HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] Expand [ws_net_paid,i_category,i_class] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index a82b90a2a9..78435aff4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -1,51 +1,59 @@ == Physical Plan == -* HashAggregate (47) -+- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin LeftAnti BuildRight (43) - :- * BroadcastHashJoin LeftAnti BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer (7) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- BroadcastExchange (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * ColumnarToRow (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.web_sales (30) - : +- ReusedExchange (33) - +- ReusedExchange (36) +* ColumnarToRow (55) ++- CometHashAggregate (54) + +- CometColumnarExchange (53) + +- RowToColumnar (52) + +- * HashAggregate (51) + +- * Project (50) + +- * BroadcastHashJoin LeftAnti BuildRight (49) + :- * BroadcastHashJoin LeftAnti BuildRight (33) + : :- * ColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometColumnarExchange (15) + : : +- RowToColumnar (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer (7) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometHashAggregate (30) + : +- CometColumnarExchange (29) + : +- RowToColumnar (28) + : +- * HashAggregate (27) + : +- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * ColumnarToRow (20) + : : : +- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (21) + : +- ReusedExchange (24) + +- BroadcastExchange (48) + +- * ColumnarToRow (47) + +- CometHashAggregate (46) + +- CometColumnarExchange (45) + +- RowToColumnar (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * ColumnarToRow (36) + : : +- CometFilter (35) + : : +- CometScan parquet spark_catalog.default.web_sales (34) + : +- ReusedExchange (37) + +- ReusedExchange (40) (unknown) Scan parquet spark_catalog.default.store_sales @@ -63,7 +71,7 @@ Condition : isnotnull(ss_customer_sk#1) (3) ColumnarToRow [codegen id : 3] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -(4) ReusedExchange [Reuses operator id: 52] +(4) ReusedExchange [Reuses operator id: 60] Output [2]: [d_date_sk#4, d_date#5] (5) BroadcastHashJoin [codegen id : 3] @@ -111,16 +119,20 @@ Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#8, c_first_name#7, d_date#5] -(14) Exchange +(14) RowToColumnar Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 12] +(15) CometColumnarExchange +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [3]: [c_last_name#8, c_first_name#7, d_date#5] Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] + +(17) ColumnarToRow [codegen id : 12] +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] @@ -130,62 +142,66 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(17) CometFilter +(19) CometFilter Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] Condition : isnotnull(cs_bill_customer_sk#9) -(18) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 6] Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] -(19) ReusedExchange [Reuses operator id: 52] +(21) ReusedExchange [Reuses operator id: 60] Output [2]: [d_date_sk#12, d_date#13] -(20) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(21) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [2]: [cs_bill_customer_sk#9, d_date#13] Input [4]: [cs_bill_customer_sk#9, cs_sold_date_sk#10, d_date_sk#12, d_date#13] -(22) ReusedExchange [Reuses operator id: 10] +(24) ReusedExchange [Reuses operator id: 10] Output [3]: [c_customer_sk#14, c_first_name#15, c_last_name#16] -(23) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_bill_customer_sk#9] Right keys [1]: [c_customer_sk#14] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [3]: [c_last_name#16, c_first_name#15, d_date#13] Input [5]: [cs_bill_customer_sk#9, d_date#13, c_customer_sk#14, c_first_name#15, c_last_name#16] -(25) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 6] Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(26) Exchange +(28) RowToColumnar Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(27) HashAggregate [codegen id : 7] +(29) CometColumnarExchange +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(30) CometHashAggregate Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(28) BroadcastExchange +(31) ColumnarToRow [codegen id : 7] +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] + +(32) BroadcastExchange Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 12] +(33) BroadcastHashJoin [codegen id : 12] Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#16, ), isnull(c_last_name#16), coalesce(c_first_name#15, ), isnull(c_first_name#15), coalesce(d_date#13, 1970-01-01), isnull(d_date#13)] Join type: LeftAnti @@ -199,123 +215,131 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#18), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(31) CometFilter +(35) CometFilter Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] Condition : isnotnull(ws_bill_customer_sk#17) -(32) ColumnarToRow [codegen id : 10] +(36) ColumnarToRow [codegen id : 10] Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] -(33) ReusedExchange [Reuses operator id: 52] +(37) ReusedExchange [Reuses operator id: 60] Output [2]: [d_date_sk#20, d_date#21] -(34) BroadcastHashJoin [codegen id : 10] +(38) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#18] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 10] +(39) Project [codegen id : 10] Output [2]: [ws_bill_customer_sk#17, d_date#21] Input [4]: [ws_bill_customer_sk#17, ws_sold_date_sk#18, d_date_sk#20, d_date#21] -(36) ReusedExchange [Reuses operator id: 10] +(40) ReusedExchange [Reuses operator id: 10] Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] -(37) BroadcastHashJoin [codegen id : 10] +(41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_bill_customer_sk#17] Right keys [1]: [c_customer_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(42) Project [codegen id : 10] Output [3]: [c_last_name#24, c_first_name#23, d_date#21] Input [5]: [ws_bill_customer_sk#17, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] -(39) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 10] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(40) Exchange +(44) RowToColumnar +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] + +(45) CometColumnarExchange Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) HashAggregate [codegen id : 11] +(46) CometHashAggregate Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(42) BroadcastExchange +(47) ColumnarToRow [codegen id : 11] +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] + +(48) BroadcastExchange Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 12] +(49) BroadcastHashJoin [codegen id : 12] Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] Join type: LeftAnti Join condition: None -(44) Project [codegen id : 12] +(50) Project [codegen id : 12] Output: [] Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -(45) HashAggregate [codegen id : 12] +(51) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#25] Results [1]: [count#26] -(46) Exchange +(52) RowToColumnar Input [1]: [count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(47) HashAggregate [codegen id : 13] +(53) CometColumnarExchange +Input [1]: [count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(54) CometHashAggregate Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#27] -Results [1]: [count(1)#27 AS count(1)#28] + +(55) ColumnarToRow [codegen id : 13] +Input [1]: [count(1)#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (60) ++- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) +(57) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#28] +Condition : (((isnotnull(d_month_seq#28) AND (d_month_seq#28 >= 1200)) AND (d_month_seq#28 <= 1211)) AND isnotnull(d_date_sk#4)) -(50) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +(58) CometProject +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#28] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(59) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(52) BroadcastExchange +(60) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 +Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 34 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index 315afe6602..28771f9d42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -1,81 +1,89 @@ WholeStageCodegen (13) - HashAggregate [count] [count(1),count(1),count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (12) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) + CometHashAggregate [count] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (12) + HashAggregate [count,count] + Project + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #6 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #8 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index 26821dfd2a..0794d36e95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt @@ -1,186 +1,202 @@ == Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (182) -:- * BroadcastNestedLoopJoin Inner BuildRight (160) -: :- * BroadcastNestedLoopJoin Inner BuildRight (138) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (116) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (94) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (72) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : : : : : :- * HashAggregate (28) -: : : : : : : +- Exchange (27) -: : : : : : : +- * HashAggregate (26) -: : : : : : : +- * Project (25) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (24) -: : : : : : : :- * Project (18) -: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) -: : : : : : : : :- * Project (11) -: : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (10) -: : : : : : : : : :- * ColumnarToRow (4) -: : : : : : : : : : +- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- BroadcastExchange (9) -: : : : : : : : : +- * ColumnarToRow (8) -: : : : : : : : : +- CometProject (7) -: : : : : : : : : +- CometFilter (6) -: : : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (5) -: : : : : : : : +- BroadcastExchange (16) -: : : : : : : : +- * ColumnarToRow (15) -: : : : : : : : +- CometProject (14) -: : : : : : : : +- CometFilter (13) -: : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (12) -: : : : : : : +- BroadcastExchange (23) -: : : : : : : +- * ColumnarToRow (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometFilter (20) -: : : : : : : +- CometScan parquet spark_catalog.default.store (19) -: : : : : : +- BroadcastExchange (49) -: : : : : : +- * HashAggregate (48) -: : : : : : +- Exchange (47) -: : : : : : +- * HashAggregate (46) -: : : : : : +- * Project (45) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (44) -: : : : : : :- * Project (42) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) -: : : : : : : :- * Project (35) -: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) -: : : : : : : : :- * ColumnarToRow (32) -: : : : : : : : : +- CometProject (31) -: : : : : : : : : +- CometFilter (30) -: : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) -: : : : : : : : +- ReusedExchange (33) -: : : : : : : +- BroadcastExchange (40) -: : : : : : : +- * ColumnarToRow (39) -: : : : : : : +- CometProject (38) -: : : : : : : +- CometFilter (37) -: : : : : : : +- CometScan parquet spark_catalog.default.time_dim (36) -: : : : : : +- ReusedExchange (43) -: : : : : +- BroadcastExchange (71) -: : : : : +- * HashAggregate (70) -: : : : : +- Exchange (69) -: : : : : +- * HashAggregate (68) -: : : : : +- * Project (67) -: : : : : +- * BroadcastHashJoin Inner BuildRight (66) -: : : : : :- * Project (64) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (63) -: : : : : : :- * Project (57) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) -: : : : : : : :- * ColumnarToRow (54) -: : : : : : : : +- CometProject (53) -: : : : : : : : +- CometFilter (52) -: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (51) -: : : : : : : +- ReusedExchange (55) -: : : : : : +- BroadcastExchange (62) -: : : : : : +- * ColumnarToRow (61) -: : : : : : +- CometProject (60) -: : : : : : +- CometFilter (59) -: : : : : : +- CometScan parquet spark_catalog.default.time_dim (58) -: : : : : +- ReusedExchange (65) -: : : : +- BroadcastExchange (93) -: : : : +- * HashAggregate (92) -: : : : +- Exchange (91) -: : : : +- * HashAggregate (90) -: : : : +- * Project (89) -: : : : +- * BroadcastHashJoin Inner BuildRight (88) -: : : : :- * Project (86) -: : : : : +- * BroadcastHashJoin Inner BuildRight (85) -: : : : : :- * Project (79) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (78) -: : : : : : :- * ColumnarToRow (76) -: : : : : : : +- CometProject (75) -: : : : : : : +- CometFilter (74) -: : : : : : : +- CometScan parquet spark_catalog.default.store_sales (73) -: : : : : : +- ReusedExchange (77) -: : : : : +- BroadcastExchange (84) -: : : : : +- * ColumnarToRow (83) -: : : : : +- CometProject (82) -: : : : : +- CometFilter (81) -: : : : : +- CometScan parquet spark_catalog.default.time_dim (80) -: : : : +- ReusedExchange (87) -: : : +- BroadcastExchange (115) -: : : +- * HashAggregate (114) -: : : +- Exchange (113) -: : : +- * HashAggregate (112) -: : : +- * Project (111) -: : : +- * BroadcastHashJoin Inner BuildRight (110) -: : : :- * Project (108) -: : : : +- * BroadcastHashJoin Inner BuildRight (107) -: : : : :- * Project (101) -: : : : : +- * BroadcastHashJoin Inner BuildRight (100) -: : : : : :- * ColumnarToRow (98) -: : : : : : +- CometProject (97) -: : : : : : +- CometFilter (96) -: : : : : : +- CometScan parquet spark_catalog.default.store_sales (95) -: : : : : +- ReusedExchange (99) -: : : : +- BroadcastExchange (106) -: : : : +- * ColumnarToRow (105) -: : : : +- CometProject (104) -: : : : +- CometFilter (103) -: : : : +- CometScan parquet spark_catalog.default.time_dim (102) -: : : +- ReusedExchange (109) -: : +- BroadcastExchange (137) -: : +- * HashAggregate (136) -: : +- Exchange (135) -: : +- * HashAggregate (134) -: : +- * Project (133) -: : +- * BroadcastHashJoin Inner BuildRight (132) -: : :- * Project (130) -: : : +- * BroadcastHashJoin Inner BuildRight (129) -: : : :- * Project (123) -: : : : +- * BroadcastHashJoin Inner BuildRight (122) -: : : : :- * ColumnarToRow (120) -: : : : : +- CometProject (119) -: : : : : +- CometFilter (118) -: : : : : +- CometScan parquet spark_catalog.default.store_sales (117) -: : : : +- ReusedExchange (121) -: : : +- BroadcastExchange (128) -: : : +- * ColumnarToRow (127) -: : : +- CometProject (126) -: : : +- CometFilter (125) -: : : +- CometScan parquet spark_catalog.default.time_dim (124) -: : +- ReusedExchange (131) -: +- BroadcastExchange (159) -: +- * HashAggregate (158) -: +- Exchange (157) -: +- * HashAggregate (156) -: +- * Project (155) -: +- * BroadcastHashJoin Inner BuildRight (154) -: :- * Project (152) -: : +- * BroadcastHashJoin Inner BuildRight (151) -: : :- * Project (145) -: : : +- * BroadcastHashJoin Inner BuildRight (144) -: : : :- * ColumnarToRow (142) -: : : : +- CometProject (141) -: : : : +- CometFilter (140) -: : : : +- CometScan parquet spark_catalog.default.store_sales (139) -: : : +- ReusedExchange (143) -: : +- BroadcastExchange (150) -: : +- * ColumnarToRow (149) -: : +- CometProject (148) -: : +- CometFilter (147) -: : +- CometScan parquet spark_catalog.default.time_dim (146) -: +- ReusedExchange (153) -+- BroadcastExchange (181) - +- * HashAggregate (180) - +- Exchange (179) - +- * HashAggregate (178) - +- * Project (177) - +- * BroadcastHashJoin Inner BuildRight (176) - :- * Project (174) - : +- * BroadcastHashJoin Inner BuildRight (173) - : :- * Project (167) - : : +- * BroadcastHashJoin Inner BuildRight (166) - : : :- * ColumnarToRow (164) - : : : +- CometProject (163) - : : : +- CometFilter (162) - : : : +- CometScan parquet spark_catalog.default.store_sales (161) - : : +- ReusedExchange (165) - : +- BroadcastExchange (172) - : +- * ColumnarToRow (171) - : +- CometProject (170) - : +- CometFilter (169) - : +- CometScan parquet spark_catalog.default.time_dim (168) - +- ReusedExchange (175) +* BroadcastNestedLoopJoin Inner BuildRight (198) +:- * BroadcastNestedLoopJoin Inner BuildRight (174) +: :- * BroadcastNestedLoopJoin Inner BuildRight (150) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (126) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (102) +: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (78) +: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (54) +: : : : : : :- * ColumnarToRow (30) +: : : : : : : +- CometHashAggregate (29) +: : : : : : : +- CometColumnarExchange (28) +: : : : : : : +- RowToColumnar (27) +: : : : : : : +- * HashAggregate (26) +: : : : : : : +- * Project (25) +: : : : : : : +- * BroadcastHashJoin Inner BuildRight (24) +: : : : : : : :- * Project (18) +: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) +: : : : : : : : :- * Project (11) +: : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (10) +: : : : : : : : : :- * ColumnarToRow (4) +: : : : : : : : : : +- CometProject (3) +: : : : : : : : : : +- CometFilter (2) +: : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : : : : : : +- BroadcastExchange (9) +: : : : : : : : : +- * ColumnarToRow (8) +: : : : : : : : : +- CometProject (7) +: : : : : : : : : +- CometFilter (6) +: : : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (5) +: : : : : : : : +- BroadcastExchange (16) +: : : : : : : : +- * ColumnarToRow (15) +: : : : : : : : +- CometProject (14) +: : : : : : : : +- CometFilter (13) +: : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (12) +: : : : : : : +- BroadcastExchange (23) +: : : : : : : +- * ColumnarToRow (22) +: : : : : : : +- CometProject (21) +: : : : : : : +- CometFilter (20) +: : : : : : : +- CometScan parquet spark_catalog.default.store (19) +: : : : : : +- BroadcastExchange (53) +: : : : : : +- * ColumnarToRow (52) +: : : : : : +- CometHashAggregate (51) +: : : : : : +- CometColumnarExchange (50) +: : : : : : +- RowToColumnar (49) +: : : : : : +- * HashAggregate (48) +: : : : : : +- * Project (47) +: : : : : : +- * BroadcastHashJoin Inner BuildRight (46) +: : : : : : :- * Project (44) +: : : : : : : +- * BroadcastHashJoin Inner BuildRight (43) +: : : : : : : :- * Project (37) +: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (36) +: : : : : : : : :- * ColumnarToRow (34) +: : : : : : : : : +- CometProject (33) +: : : : : : : : : +- CometFilter (32) +: : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (31) +: : : : : : : : +- ReusedExchange (35) +: : : : : : : +- BroadcastExchange (42) +: : : : : : : +- * ColumnarToRow (41) +: : : : : : : +- CometProject (40) +: : : : : : : +- CometFilter (39) +: : : : : : : +- CometScan parquet spark_catalog.default.time_dim (38) +: : : : : : +- ReusedExchange (45) +: : : : : +- BroadcastExchange (77) +: : : : : +- * ColumnarToRow (76) +: : : : : +- CometHashAggregate (75) +: : : : : +- CometColumnarExchange (74) +: : : : : +- RowToColumnar (73) +: : : : : +- * HashAggregate (72) +: : : : : +- * Project (71) +: : : : : +- * BroadcastHashJoin Inner BuildRight (70) +: : : : : :- * Project (68) +: : : : : : +- * BroadcastHashJoin Inner BuildRight (67) +: : : : : : :- * Project (61) +: : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) +: : : : : : : :- * ColumnarToRow (58) +: : : : : : : : +- CometProject (57) +: : : : : : : : +- CometFilter (56) +: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (55) +: : : : : : : +- ReusedExchange (59) +: : : : : : +- BroadcastExchange (66) +: : : : : : +- * ColumnarToRow (65) +: : : : : : +- CometProject (64) +: : : : : : +- CometFilter (63) +: : : : : : +- CometScan parquet spark_catalog.default.time_dim (62) +: : : : : +- ReusedExchange (69) +: : : : +- BroadcastExchange (101) +: : : : +- * ColumnarToRow (100) +: : : : +- CometHashAggregate (99) +: : : : +- CometColumnarExchange (98) +: : : : +- RowToColumnar (97) +: : : : +- * HashAggregate (96) +: : : : +- * Project (95) +: : : : +- * BroadcastHashJoin Inner BuildRight (94) +: : : : :- * Project (92) +: : : : : +- * BroadcastHashJoin Inner BuildRight (91) +: : : : : :- * Project (85) +: : : : : : +- * BroadcastHashJoin Inner BuildRight (84) +: : : : : : :- * ColumnarToRow (82) +: : : : : : : +- CometProject (81) +: : : : : : : +- CometFilter (80) +: : : : : : : +- CometScan parquet spark_catalog.default.store_sales (79) +: : : : : : +- ReusedExchange (83) +: : : : : +- BroadcastExchange (90) +: : : : : +- * ColumnarToRow (89) +: : : : : +- CometProject (88) +: : : : : +- CometFilter (87) +: : : : : +- CometScan parquet spark_catalog.default.time_dim (86) +: : : : +- ReusedExchange (93) +: : : +- BroadcastExchange (125) +: : : +- * ColumnarToRow (124) +: : : +- CometHashAggregate (123) +: : : +- CometColumnarExchange (122) +: : : +- RowToColumnar (121) +: : : +- * HashAggregate (120) +: : : +- * Project (119) +: : : +- * BroadcastHashJoin Inner BuildRight (118) +: : : :- * Project (116) +: : : : +- * BroadcastHashJoin Inner BuildRight (115) +: : : : :- * Project (109) +: : : : : +- * BroadcastHashJoin Inner BuildRight (108) +: : : : : :- * ColumnarToRow (106) +: : : : : : +- CometProject (105) +: : : : : : +- CometFilter (104) +: : : : : : +- CometScan parquet spark_catalog.default.store_sales (103) +: : : : : +- ReusedExchange (107) +: : : : +- BroadcastExchange (114) +: : : : +- * ColumnarToRow (113) +: : : : +- CometProject (112) +: : : : +- CometFilter (111) +: : : : +- CometScan parquet spark_catalog.default.time_dim (110) +: : : +- ReusedExchange (117) +: : +- BroadcastExchange (149) +: : +- * ColumnarToRow (148) +: : +- CometHashAggregate (147) +: : +- CometColumnarExchange (146) +: : +- RowToColumnar (145) +: : +- * HashAggregate (144) +: : +- * Project (143) +: : +- * BroadcastHashJoin Inner BuildRight (142) +: : :- * Project (140) +: : : +- * BroadcastHashJoin Inner BuildRight (139) +: : : :- * Project (133) +: : : : +- * BroadcastHashJoin Inner BuildRight (132) +: : : : :- * ColumnarToRow (130) +: : : : : +- CometProject (129) +: : : : : +- CometFilter (128) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (127) +: : : : +- ReusedExchange (131) +: : : +- BroadcastExchange (138) +: : : +- * ColumnarToRow (137) +: : : +- CometProject (136) +: : : +- CometFilter (135) +: : : +- CometScan parquet spark_catalog.default.time_dim (134) +: : +- ReusedExchange (141) +: +- BroadcastExchange (173) +: +- * ColumnarToRow (172) +: +- CometHashAggregate (171) +: +- CometColumnarExchange (170) +: +- RowToColumnar (169) +: +- * HashAggregate (168) +: +- * Project (167) +: +- * BroadcastHashJoin Inner BuildRight (166) +: :- * Project (164) +: : +- * BroadcastHashJoin Inner BuildRight (163) +: : :- * Project (157) +: : : +- * BroadcastHashJoin Inner BuildRight (156) +: : : :- * ColumnarToRow (154) +: : : : +- CometProject (153) +: : : : +- CometFilter (152) +: : : : +- CometScan parquet spark_catalog.default.store_sales (151) +: : : +- ReusedExchange (155) +: : +- BroadcastExchange (162) +: : +- * ColumnarToRow (161) +: : +- CometProject (160) +: : +- CometFilter (159) +: : +- CometScan parquet spark_catalog.default.time_dim (158) +: +- ReusedExchange (165) ++- BroadcastExchange (197) + +- * ColumnarToRow (196) + +- CometHashAggregate (195) + +- CometColumnarExchange (194) + +- RowToColumnar (193) + +- * HashAggregate (192) + +- * Project (191) + +- * BroadcastHashJoin Inner BuildRight (190) + :- * Project (188) + : +- * BroadcastHashJoin Inner BuildRight (187) + : :- * Project (181) + : : +- * BroadcastHashJoin Inner BuildRight (180) + : : :- * ColumnarToRow (178) + : : : +- CometProject (177) + : : : +- CometFilter (176) + : : : +- CometScan parquet spark_catalog.default.store_sales (175) + : : +- ReusedExchange (179) + : +- BroadcastExchange (186) + : +- * ColumnarToRow (185) + : +- CometProject (184) + : +- CometFilter (183) + : +- CometScan parquet spark_catalog.default.time_dim (182) + +- ReusedExchange (189) (unknown) Scan parquet spark_catalog.default.store_sales @@ -304,728 +320,760 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#13] Results [1]: [count#14] -(27) Exchange +(27) RowToColumnar Input [1]: [count#14] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(28) HashAggregate [codegen id : 40] +(28) CometColumnarExchange +Input [1]: [count#14] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(29) CometHashAggregate Input [1]: [count#14] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#15] -Results [1]: [count(1)#15 AS h8_30_to_9#16] + +(30) ColumnarToRow [codegen id : 40] +Input [1]: [h8_30_to_9#15] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] +Output [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_hdemo_sk#18) AND isnotnull(ss_sold_time_sk#17)) AND isnotnull(ss_store_sk#19)) +(32) CometFilter +Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] +Condition : ((isnotnull(ss_hdemo_sk#17) AND isnotnull(ss_sold_time_sk#16)) AND isnotnull(ss_store_sk#18)) -(31) CometProject -Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] -Arguments: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19], [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] +(33) CometProject +Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] +Arguments: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18], [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18] -(32) ColumnarToRow [codegen id : 8] -Input [3]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] +(34) ColumnarToRow [codegen id : 8] +Input [3]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18] -(33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#21] +(35) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#20] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#18] -Right keys [1]: [hd_demo_sk#21] +(36) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_hdemo_sk#17] +Right keys [1]: [hd_demo_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 8] -Output [2]: [ss_sold_time_sk#17, ss_store_sk#19] -Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, hd_demo_sk#21] +(37) Project [codegen id : 8] +Output [2]: [ss_sold_time_sk#16, ss_store_sk#18] +Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, hd_demo_sk#20] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#22, t_hour#23, t_minute#24] +Output [3]: [t_time_sk#21, t_hour#22, t_minute#23] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(37) CometFilter -Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] -Condition : ((((isnotnull(t_hour#23) AND isnotnull(t_minute#24)) AND (t_hour#23 = 9)) AND (t_minute#24 < 30)) AND isnotnull(t_time_sk#22)) +(39) CometFilter +Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Condition : ((((isnotnull(t_hour#22) AND isnotnull(t_minute#23)) AND (t_hour#22 = 9)) AND (t_minute#23 < 30)) AND isnotnull(t_time_sk#21)) -(38) CometProject -Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] -Arguments: [t_time_sk#22], [t_time_sk#22] +(40) CometProject +Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Arguments: [t_time_sk#21], [t_time_sk#21] -(39) ColumnarToRow [codegen id : 6] -Input [1]: [t_time_sk#22] +(41) ColumnarToRow [codegen id : 6] +Input [1]: [t_time_sk#21] -(40) BroadcastExchange -Input [1]: [t_time_sk#22] +(42) BroadcastExchange +Input [1]: [t_time_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#17] -Right keys [1]: [t_time_sk#22] +(43) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_time_sk#16] +Right keys [1]: [t_time_sk#21] Join type: Inner Join condition: None -(42) Project [codegen id : 8] -Output [1]: [ss_store_sk#19] -Input [3]: [ss_sold_time_sk#17, ss_store_sk#19, t_time_sk#22] +(44) Project [codegen id : 8] +Output [1]: [ss_store_sk#18] +Input [3]: [ss_sold_time_sk#16, ss_store_sk#18, t_time_sk#21] -(43) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#25] +(45) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#24] -(44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#19] -Right keys [1]: [s_store_sk#25] +(46) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_store_sk#18] +Right keys [1]: [s_store_sk#24] Join type: Inner Join condition: None -(45) Project [codegen id : 8] +(47) Project [codegen id : 8] Output: [] -Input [2]: [ss_store_sk#19, s_store_sk#25] +Input [2]: [ss_store_sk#18, s_store_sk#24] -(46) HashAggregate [codegen id : 8] +(48) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#26] -Results [1]: [count#27] +Aggregate Attributes [1]: [count#25] +Results [1]: [count#26] -(47) Exchange -Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(49) RowToColumnar +Input [1]: [count#26] -(48) HashAggregate [codegen id : 9] -Input [1]: [count#27] +(50) CometColumnarExchange +Input [1]: [count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(51) CometHashAggregate +Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#28] -Results [1]: [count(1)#28 AS h9_to_9_30#29] -(49) BroadcastExchange -Input [1]: [h9_to_9_30#29] +(52) ColumnarToRow [codegen id : 9] +Input [1]: [h9_to_9_30#27] + +(53) BroadcastExchange +Input [1]: [h9_to_9_30#27] Arguments: IdentityBroadcastMode, [plan_id=7] -(50) BroadcastNestedLoopJoin [codegen id : 40] +(54) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] +Output [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(52) CometFilter -Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_hdemo_sk#31) AND isnotnull(ss_sold_time_sk#30)) AND isnotnull(ss_store_sk#32)) +(56) CometFilter +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] +Condition : ((isnotnull(ss_hdemo_sk#29) AND isnotnull(ss_sold_time_sk#28)) AND isnotnull(ss_store_sk#30)) -(53) CometProject -Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] -Arguments: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32], [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] +(57) CometProject +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] +Arguments: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30], [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30] -(54) ColumnarToRow [codegen id : 13] -Input [3]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] +(58) ColumnarToRow [codegen id : 13] +Input [3]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30] -(55) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#34] +(59) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#32] -(56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#31] -Right keys [1]: [hd_demo_sk#34] +(60) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_hdemo_sk#29] +Right keys [1]: [hd_demo_sk#32] Join type: Inner Join condition: None -(57) Project [codegen id : 13] -Output [2]: [ss_sold_time_sk#30, ss_store_sk#32] -Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, hd_demo_sk#34] +(61) Project [codegen id : 13] +Output [2]: [ss_sold_time_sk#28, ss_store_sk#30] +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, hd_demo_sk#32] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#35, t_hour#36, t_minute#37] +Output [3]: [t_time_sk#33, t_hour#34, t_minute#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(59) CometFilter -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Condition : ((((isnotnull(t_hour#36) AND isnotnull(t_minute#37)) AND (t_hour#36 = 9)) AND (t_minute#37 >= 30)) AND isnotnull(t_time_sk#35)) +(63) CometFilter +Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Condition : ((((isnotnull(t_hour#34) AND isnotnull(t_minute#35)) AND (t_hour#34 = 9)) AND (t_minute#35 >= 30)) AND isnotnull(t_time_sk#33)) -(60) CometProject -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [t_time_sk#35], [t_time_sk#35] +(64) CometProject +Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Arguments: [t_time_sk#33], [t_time_sk#33] -(61) ColumnarToRow [codegen id : 11] -Input [1]: [t_time_sk#35] +(65) ColumnarToRow [codegen id : 11] +Input [1]: [t_time_sk#33] -(62) BroadcastExchange -Input [1]: [t_time_sk#35] +(66) BroadcastExchange +Input [1]: [t_time_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#30] -Right keys [1]: [t_time_sk#35] +(67) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_sold_time_sk#28] +Right keys [1]: [t_time_sk#33] Join type: Inner Join condition: None -(64) Project [codegen id : 13] -Output [1]: [ss_store_sk#32] -Input [3]: [ss_sold_time_sk#30, ss_store_sk#32, t_time_sk#35] +(68) Project [codegen id : 13] +Output [1]: [ss_store_sk#30] +Input [3]: [ss_sold_time_sk#28, ss_store_sk#30, t_time_sk#33] -(65) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#38] +(69) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#36] -(66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#32] -Right keys [1]: [s_store_sk#38] +(70) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_store_sk#30] +Right keys [1]: [s_store_sk#36] Join type: Inner Join condition: None -(67) Project [codegen id : 13] +(71) Project [codegen id : 13] Output: [] -Input [2]: [ss_store_sk#32, s_store_sk#38] +Input [2]: [ss_store_sk#30, s_store_sk#36] -(68) HashAggregate [codegen id : 13] +(72) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#39] -Results [1]: [count#40] +Aggregate Attributes [1]: [count#37] +Results [1]: [count#38] + +(73) RowToColumnar +Input [1]: [count#38] -(69) Exchange -Input [1]: [count#40] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +(74) CometColumnarExchange +Input [1]: [count#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(70) HashAggregate [codegen id : 14] -Input [1]: [count#40] +(75) CometHashAggregate +Input [1]: [count#38] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#41] -Results [1]: [count(1)#41 AS h9_30_to_10#42] -(71) BroadcastExchange -Input [1]: [h9_30_to_10#42] +(76) ColumnarToRow [codegen id : 14] +Input [1]: [h9_30_to_10#39] + +(77) BroadcastExchange +Input [1]: [h9_30_to_10#39] Arguments: IdentityBroadcastMode, [plan_id=10] -(72) BroadcastNestedLoopJoin [codegen id : 40] +(78) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] +Output [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(74) CometFilter -Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_hdemo_sk#44) AND isnotnull(ss_sold_time_sk#43)) AND isnotnull(ss_store_sk#45)) +(80) CometFilter +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] +Condition : ((isnotnull(ss_hdemo_sk#41) AND isnotnull(ss_sold_time_sk#40)) AND isnotnull(ss_store_sk#42)) -(75) CometProject -Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] -Arguments: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45], [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] +(81) CometProject +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] +Arguments: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42], [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42] -(76) ColumnarToRow [codegen id : 18] -Input [3]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] +(82) ColumnarToRow [codegen id : 18] +Input [3]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42] -(77) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#47] +(83) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#44] -(78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#44] -Right keys [1]: [hd_demo_sk#47] +(84) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_hdemo_sk#41] +Right keys [1]: [hd_demo_sk#44] Join type: Inner Join condition: None -(79) Project [codegen id : 18] -Output [2]: [ss_sold_time_sk#43, ss_store_sk#45] -Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, hd_demo_sk#47] +(85) Project [codegen id : 18] +Output [2]: [ss_sold_time_sk#40, ss_store_sk#42] +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, hd_demo_sk#44] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#48, t_hour#49, t_minute#50] +Output [3]: [t_time_sk#45, t_hour#46, t_minute#47] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(81) CometFilter -Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] -Condition : ((((isnotnull(t_hour#49) AND isnotnull(t_minute#50)) AND (t_hour#49 = 10)) AND (t_minute#50 < 30)) AND isnotnull(t_time_sk#48)) +(87) CometFilter +Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Condition : ((((isnotnull(t_hour#46) AND isnotnull(t_minute#47)) AND (t_hour#46 = 10)) AND (t_minute#47 < 30)) AND isnotnull(t_time_sk#45)) -(82) CometProject -Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] -Arguments: [t_time_sk#48], [t_time_sk#48] +(88) CometProject +Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Arguments: [t_time_sk#45], [t_time_sk#45] -(83) ColumnarToRow [codegen id : 16] -Input [1]: [t_time_sk#48] +(89) ColumnarToRow [codegen id : 16] +Input [1]: [t_time_sk#45] -(84) BroadcastExchange -Input [1]: [t_time_sk#48] +(90) BroadcastExchange +Input [1]: [t_time_sk#45] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#43] -Right keys [1]: [t_time_sk#48] +(91) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_sold_time_sk#40] +Right keys [1]: [t_time_sk#45] Join type: Inner Join condition: None -(86) Project [codegen id : 18] -Output [1]: [ss_store_sk#45] -Input [3]: [ss_sold_time_sk#43, ss_store_sk#45, t_time_sk#48] +(92) Project [codegen id : 18] +Output [1]: [ss_store_sk#42] +Input [3]: [ss_sold_time_sk#40, ss_store_sk#42, t_time_sk#45] -(87) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#51] +(93) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#48] -(88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#45] -Right keys [1]: [s_store_sk#51] +(94) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_store_sk#42] +Right keys [1]: [s_store_sk#48] Join type: Inner Join condition: None -(89) Project [codegen id : 18] +(95) Project [codegen id : 18] Output: [] -Input [2]: [ss_store_sk#45, s_store_sk#51] +Input [2]: [ss_store_sk#42, s_store_sk#48] -(90) HashAggregate [codegen id : 18] +(96) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#52] -Results [1]: [count#53] +Aggregate Attributes [1]: [count#49] +Results [1]: [count#50] + +(97) RowToColumnar +Input [1]: [count#50] -(91) Exchange -Input [1]: [count#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(98) CometColumnarExchange +Input [1]: [count#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(92) HashAggregate [codegen id : 19] -Input [1]: [count#53] +(99) CometHashAggregate +Input [1]: [count#50] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#54] -Results [1]: [count(1)#54 AS h10_to_10_30#55] -(93) BroadcastExchange -Input [1]: [h10_to_10_30#55] +(100) ColumnarToRow [codegen id : 19] +Input [1]: [h10_to_10_30#51] + +(101) BroadcastExchange +Input [1]: [h10_to_10_30#51] Arguments: IdentityBroadcastMode, [plan_id=13] -(94) BroadcastNestedLoopJoin [codegen id : 40] +(102) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] +Output [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(96) CometFilter -Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_hdemo_sk#57) AND isnotnull(ss_sold_time_sk#56)) AND isnotnull(ss_store_sk#58)) +(104) CometFilter +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] +Condition : ((isnotnull(ss_hdemo_sk#53) AND isnotnull(ss_sold_time_sk#52)) AND isnotnull(ss_store_sk#54)) -(97) CometProject -Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] -Arguments: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58], [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] +(105) CometProject +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] +Arguments: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54], [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54] -(98) ColumnarToRow [codegen id : 23] -Input [3]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] +(106) ColumnarToRow [codegen id : 23] +Input [3]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54] -(99) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#60] +(107) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#56] -(100) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#57] -Right keys [1]: [hd_demo_sk#60] +(108) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [ss_hdemo_sk#53] +Right keys [1]: [hd_demo_sk#56] Join type: Inner Join condition: None -(101) Project [codegen id : 23] -Output [2]: [ss_sold_time_sk#56, ss_store_sk#58] -Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, hd_demo_sk#60] +(109) Project [codegen id : 23] +Output [2]: [ss_sold_time_sk#52, ss_store_sk#54] +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, hd_demo_sk#56] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#61, t_hour#62, t_minute#63] +Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(103) CometFilter -Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] -Condition : ((((isnotnull(t_hour#62) AND isnotnull(t_minute#63)) AND (t_hour#62 = 10)) AND (t_minute#63 >= 30)) AND isnotnull(t_time_sk#61)) +(111) CometFilter +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Condition : ((((isnotnull(t_hour#58) AND isnotnull(t_minute#59)) AND (t_hour#58 = 10)) AND (t_minute#59 >= 30)) AND isnotnull(t_time_sk#57)) -(104) CometProject -Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] -Arguments: [t_time_sk#61], [t_time_sk#61] +(112) CometProject +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Arguments: [t_time_sk#57], [t_time_sk#57] -(105) ColumnarToRow [codegen id : 21] -Input [1]: [t_time_sk#61] +(113) ColumnarToRow [codegen id : 21] +Input [1]: [t_time_sk#57] -(106) BroadcastExchange -Input [1]: [t_time_sk#61] +(114) BroadcastExchange +Input [1]: [t_time_sk#57] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -(107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#56] -Right keys [1]: [t_time_sk#61] +(115) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [ss_sold_time_sk#52] +Right keys [1]: [t_time_sk#57] Join type: Inner Join condition: None -(108) Project [codegen id : 23] -Output [1]: [ss_store_sk#58] -Input [3]: [ss_sold_time_sk#56, ss_store_sk#58, t_time_sk#61] +(116) Project [codegen id : 23] +Output [1]: [ss_store_sk#54] +Input [3]: [ss_sold_time_sk#52, ss_store_sk#54, t_time_sk#57] -(109) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#64] +(117) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#60] -(110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#58] -Right keys [1]: [s_store_sk#64] +(118) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#60] Join type: Inner Join condition: None -(111) Project [codegen id : 23] +(119) Project [codegen id : 23] Output: [] -Input [2]: [ss_store_sk#58, s_store_sk#64] +Input [2]: [ss_store_sk#54, s_store_sk#60] -(112) HashAggregate [codegen id : 23] +(120) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#65] -Results [1]: [count#66] +Aggregate Attributes [1]: [count#61] +Results [1]: [count#62] -(113) Exchange -Input [1]: [count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] +(121) RowToColumnar +Input [1]: [count#62] -(114) HashAggregate [codegen id : 24] -Input [1]: [count#66] +(122) CometColumnarExchange +Input [1]: [count#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] + +(123) CometHashAggregate +Input [1]: [count#62] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#67] -Results [1]: [count(1)#67 AS h10_30_to_11#68] -(115) BroadcastExchange -Input [1]: [h10_30_to_11#68] +(124) ColumnarToRow [codegen id : 24] +Input [1]: [h10_30_to_11#63] + +(125) BroadcastExchange +Input [1]: [h10_30_to_11#63] Arguments: IdentityBroadcastMode, [plan_id=16] -(116) BroadcastNestedLoopJoin [codegen id : 40] +(126) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Output [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(118) CometFilter -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) +(128) CometFilter +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] +Condition : ((isnotnull(ss_hdemo_sk#65) AND isnotnull(ss_sold_time_sk#64)) AND isnotnull(ss_store_sk#66)) -(119) CometProject -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] -Arguments: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71], [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +(129) CometProject +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] +Arguments: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66], [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66] -(120) ColumnarToRow [codegen id : 28] -Input [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +(130) ColumnarToRow [codegen id : 28] +Input [3]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66] -(121) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#73] +(131) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#68] -(122) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#70] -Right keys [1]: [hd_demo_sk#73] +(132) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [ss_hdemo_sk#65] +Right keys [1]: [hd_demo_sk#68] Join type: Inner Join condition: None -(123) Project [codegen id : 28] -Output [2]: [ss_sold_time_sk#69, ss_store_sk#71] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, hd_demo_sk#73] +(133) Project [codegen id : 28] +Output [2]: [ss_sold_time_sk#64, ss_store_sk#66] +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, hd_demo_sk#68] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Output [3]: [t_time_sk#69, t_hour#70, t_minute#71] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(125) CometFilter -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] -Condition : ((((isnotnull(t_hour#75) AND isnotnull(t_minute#76)) AND (t_hour#75 = 11)) AND (t_minute#76 < 30)) AND isnotnull(t_time_sk#74)) +(135) CometFilter +Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Condition : ((((isnotnull(t_hour#70) AND isnotnull(t_minute#71)) AND (t_hour#70 = 11)) AND (t_minute#71 < 30)) AND isnotnull(t_time_sk#69)) -(126) CometProject -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] -Arguments: [t_time_sk#74], [t_time_sk#74] +(136) CometProject +Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Arguments: [t_time_sk#69], [t_time_sk#69] -(127) ColumnarToRow [codegen id : 26] -Input [1]: [t_time_sk#74] +(137) ColumnarToRow [codegen id : 26] +Input [1]: [t_time_sk#69] -(128) BroadcastExchange -Input [1]: [t_time_sk#74] +(138) BroadcastExchange +Input [1]: [t_time_sk#69] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] -(129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#69] -Right keys [1]: [t_time_sk#74] +(139) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [ss_sold_time_sk#64] +Right keys [1]: [t_time_sk#69] Join type: Inner Join condition: None -(130) Project [codegen id : 28] -Output [1]: [ss_store_sk#71] -Input [3]: [ss_sold_time_sk#69, ss_store_sk#71, t_time_sk#74] +(140) Project [codegen id : 28] +Output [1]: [ss_store_sk#66] +Input [3]: [ss_sold_time_sk#64, ss_store_sk#66, t_time_sk#69] -(131) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#77] +(141) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#72] -(132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#71] -Right keys [1]: [s_store_sk#77] +(142) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [ss_store_sk#66] +Right keys [1]: [s_store_sk#72] Join type: Inner Join condition: None -(133) Project [codegen id : 28] +(143) Project [codegen id : 28] Output: [] -Input [2]: [ss_store_sk#71, s_store_sk#77] +Input [2]: [ss_store_sk#66, s_store_sk#72] -(134) HashAggregate [codegen id : 28] +(144) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#78] -Results [1]: [count#79] +Aggregate Attributes [1]: [count#73] +Results [1]: [count#74] + +(145) RowToColumnar +Input [1]: [count#74] -(135) Exchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] +(146) CometColumnarExchange +Input [1]: [count#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(136) HashAggregate [codegen id : 29] -Input [1]: [count#79] +(147) CometHashAggregate +Input [1]: [count#74] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#80] -Results [1]: [count(1)#80 AS h11_to_11_30#81] -(137) BroadcastExchange -Input [1]: [h11_to_11_30#81] +(148) ColumnarToRow [codegen id : 29] +Input [1]: [h11_to_11_30#75] + +(149) BroadcastExchange +Input [1]: [h11_to_11_30#75] Arguments: IdentityBroadcastMode, [plan_id=19] -(138) BroadcastNestedLoopJoin [codegen id : 40] +(150) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] +Output [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(140) CometFilter -Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] -Condition : ((isnotnull(ss_hdemo_sk#83) AND isnotnull(ss_sold_time_sk#82)) AND isnotnull(ss_store_sk#84)) +(152) CometFilter +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] +Condition : ((isnotnull(ss_hdemo_sk#77) AND isnotnull(ss_sold_time_sk#76)) AND isnotnull(ss_store_sk#78)) -(141) CometProject -Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] -Arguments: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84], [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] +(153) CometProject +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] +Arguments: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78], [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] -(142) ColumnarToRow [codegen id : 33] -Input [3]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] +(154) ColumnarToRow [codegen id : 33] +Input [3]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] -(143) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#86] +(155) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#80] -(144) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#83] -Right keys [1]: [hd_demo_sk#86] +(156) BroadcastHashJoin [codegen id : 33] +Left keys [1]: [ss_hdemo_sk#77] +Right keys [1]: [hd_demo_sk#80] Join type: Inner Join condition: None -(145) Project [codegen id : 33] -Output [2]: [ss_sold_time_sk#82, ss_store_sk#84] -Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, hd_demo_sk#86] +(157) Project [codegen id : 33] +Output [2]: [ss_sold_time_sk#76, ss_store_sk#78] +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, hd_demo_sk#80] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#87, t_hour#88, t_minute#89] +Output [3]: [t_time_sk#81, t_hour#82, t_minute#83] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(147) CometFilter -Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] -Condition : ((((isnotnull(t_hour#88) AND isnotnull(t_minute#89)) AND (t_hour#88 = 11)) AND (t_minute#89 >= 30)) AND isnotnull(t_time_sk#87)) +(159) CometFilter +Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Condition : ((((isnotnull(t_hour#82) AND isnotnull(t_minute#83)) AND (t_hour#82 = 11)) AND (t_minute#83 >= 30)) AND isnotnull(t_time_sk#81)) -(148) CometProject -Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] -Arguments: [t_time_sk#87], [t_time_sk#87] +(160) CometProject +Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Arguments: [t_time_sk#81], [t_time_sk#81] -(149) ColumnarToRow [codegen id : 31] -Input [1]: [t_time_sk#87] +(161) ColumnarToRow [codegen id : 31] +Input [1]: [t_time_sk#81] -(150) BroadcastExchange -Input [1]: [t_time_sk#87] +(162) BroadcastExchange +Input [1]: [t_time_sk#81] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] -(151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#82] -Right keys [1]: [t_time_sk#87] +(163) BroadcastHashJoin [codegen id : 33] +Left keys [1]: [ss_sold_time_sk#76] +Right keys [1]: [t_time_sk#81] Join type: Inner Join condition: None -(152) Project [codegen id : 33] -Output [1]: [ss_store_sk#84] -Input [3]: [ss_sold_time_sk#82, ss_store_sk#84, t_time_sk#87] +(164) Project [codegen id : 33] +Output [1]: [ss_store_sk#78] +Input [3]: [ss_sold_time_sk#76, ss_store_sk#78, t_time_sk#81] -(153) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#90] +(165) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#84] -(154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#84] -Right keys [1]: [s_store_sk#90] +(166) BroadcastHashJoin [codegen id : 33] +Left keys [1]: [ss_store_sk#78] +Right keys [1]: [s_store_sk#84] Join type: Inner Join condition: None -(155) Project [codegen id : 33] +(167) Project [codegen id : 33] Output: [] -Input [2]: [ss_store_sk#84, s_store_sk#90] +Input [2]: [ss_store_sk#78, s_store_sk#84] -(156) HashAggregate [codegen id : 33] +(168) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#91] -Results [1]: [count#92] +Aggregate Attributes [1]: [count#85] +Results [1]: [count#86] + +(169) RowToColumnar +Input [1]: [count#86] -(157) Exchange -Input [1]: [count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=21] +(170) CometColumnarExchange +Input [1]: [count#86] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=21] -(158) HashAggregate [codegen id : 34] -Input [1]: [count#92] +(171) CometHashAggregate +Input [1]: [count#86] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#93] -Results [1]: [count(1)#93 AS h11_30_to_12#94] -(159) BroadcastExchange -Input [1]: [h11_30_to_12#94] +(172) ColumnarToRow [codegen id : 34] +Input [1]: [h11_30_to_12#87] + +(173) BroadcastExchange +Input [1]: [h11_30_to_12#87] Arguments: IdentityBroadcastMode, [plan_id=22] -(160) BroadcastNestedLoopJoin [codegen id : 40] +(174) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] +Output [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(162) CometFilter -Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] -Condition : ((isnotnull(ss_hdemo_sk#96) AND isnotnull(ss_sold_time_sk#95)) AND isnotnull(ss_store_sk#97)) +(176) CometFilter +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] +Condition : ((isnotnull(ss_hdemo_sk#89) AND isnotnull(ss_sold_time_sk#88)) AND isnotnull(ss_store_sk#90)) -(163) CometProject -Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] -Arguments: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97], [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] +(177) CometProject +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] +Arguments: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90], [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90] -(164) ColumnarToRow [codegen id : 38] -Input [3]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] +(178) ColumnarToRow [codegen id : 38] +Input [3]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90] -(165) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#99] +(179) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#92] -(166) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#96] -Right keys [1]: [hd_demo_sk#99] +(180) BroadcastHashJoin [codegen id : 38] +Left keys [1]: [ss_hdemo_sk#89] +Right keys [1]: [hd_demo_sk#92] Join type: Inner Join condition: None -(167) Project [codegen id : 38] -Output [2]: [ss_sold_time_sk#95, ss_store_sk#97] -Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, hd_demo_sk#99] +(181) Project [codegen id : 38] +Output [2]: [ss_sold_time_sk#88, ss_store_sk#90] +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, hd_demo_sk#92] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#100, t_hour#101, t_minute#102] +Output [3]: [t_time_sk#93, t_hour#94, t_minute#95] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(169) CometFilter -Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] -Condition : ((((isnotnull(t_hour#101) AND isnotnull(t_minute#102)) AND (t_hour#101 = 12)) AND (t_minute#102 < 30)) AND isnotnull(t_time_sk#100)) +(183) CometFilter +Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Condition : ((((isnotnull(t_hour#94) AND isnotnull(t_minute#95)) AND (t_hour#94 = 12)) AND (t_minute#95 < 30)) AND isnotnull(t_time_sk#93)) -(170) CometProject -Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] -Arguments: [t_time_sk#100], [t_time_sk#100] +(184) CometProject +Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Arguments: [t_time_sk#93], [t_time_sk#93] -(171) ColumnarToRow [codegen id : 36] -Input [1]: [t_time_sk#100] +(185) ColumnarToRow [codegen id : 36] +Input [1]: [t_time_sk#93] -(172) BroadcastExchange -Input [1]: [t_time_sk#100] +(186) BroadcastExchange +Input [1]: [t_time_sk#93] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23] -(173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#95] -Right keys [1]: [t_time_sk#100] +(187) BroadcastHashJoin [codegen id : 38] +Left keys [1]: [ss_sold_time_sk#88] +Right keys [1]: [t_time_sk#93] Join type: Inner Join condition: None -(174) Project [codegen id : 38] -Output [1]: [ss_store_sk#97] -Input [3]: [ss_sold_time_sk#95, ss_store_sk#97, t_time_sk#100] +(188) Project [codegen id : 38] +Output [1]: [ss_store_sk#90] +Input [3]: [ss_sold_time_sk#88, ss_store_sk#90, t_time_sk#93] -(175) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#103] +(189) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#96] -(176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#97] -Right keys [1]: [s_store_sk#103] +(190) BroadcastHashJoin [codegen id : 38] +Left keys [1]: [ss_store_sk#90] +Right keys [1]: [s_store_sk#96] Join type: Inner Join condition: None -(177) Project [codegen id : 38] +(191) Project [codegen id : 38] Output: [] -Input [2]: [ss_store_sk#97, s_store_sk#103] +Input [2]: [ss_store_sk#90, s_store_sk#96] -(178) HashAggregate [codegen id : 38] +(192) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#104] -Results [1]: [count#105] +Aggregate Attributes [1]: [count#97] +Results [1]: [count#98] -(179) Exchange -Input [1]: [count#105] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=24] +(193) RowToColumnar +Input [1]: [count#98] -(180) HashAggregate [codegen id : 39] -Input [1]: [count#105] +(194) CometColumnarExchange +Input [1]: [count#98] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=24] + +(195) CometHashAggregate +Input [1]: [count#98] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#106] -Results [1]: [count(1)#106 AS h12_to_12_30#107] -(181) BroadcastExchange -Input [1]: [h12_to_12_30#107] +(196) ColumnarToRow [codegen id : 39] +Input [1]: [h12_to_12_30#99] + +(197) BroadcastExchange +Input [1]: [h12_to_12_30#99] Arguments: IdentityBroadcastMode, [plan_id=25] -(182) BroadcastNestedLoopJoin [codegen id : 40] +(198) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index b497e0bab6..b80f512c53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -6,53 +6,123 @@ WholeStageCodegen (40) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - HashAggregate [count] [count(1),h8_30_to_9,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometHashAggregate [count] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometProject [s_store_sk] + CometFilter [s_store_name,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - HashAggregate [count] [count(1),h9_to_9_30,count] + ColumnarToRow InputAdapter - Exchange #6 - WholeStageCodegen (8) + CometHashAggregate [count] + CometColumnarExchange #6 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (14) + ColumnarToRow + InputAdapter + CometHashAggregate [count] + CometColumnarExchange #9 + RowToColumnar + WholeStageCodegen (13) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometHashAggregate [count] + CometColumnarExchange #12 + RowToColumnar + WholeStageCodegen (18) HashAggregate [count,count] Project BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -68,8 +138,8 @@ WholeStageCodegen (40) InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) + BroadcastExchange #13 + WholeStageCodegen (16) ColumnarToRow InputAdapter CometProject [t_time_sk] @@ -77,13 +147,15 @@ WholeStageCodegen (40) CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (14) - HashAggregate [count] [count(1),h9_30_to_10,count] - InputAdapter - Exchange #9 - WholeStageCodegen (13) + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (24) + ColumnarToRow + InputAdapter + CometHashAggregate [count] + CometColumnarExchange #15 + RowToColumnar + WholeStageCodegen (23) HashAggregate [count,count] Project BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -99,8 +171,8 @@ WholeStageCodegen (40) InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) + BroadcastExchange #16 + WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [t_time_sk] @@ -108,13 +180,15 @@ WholeStageCodegen (40) CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (19) - HashAggregate [count] [count(1),h10_to_10_30,count] - InputAdapter - Exchange #12 - WholeStageCodegen (18) + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (29) + ColumnarToRow + InputAdapter + CometHashAggregate [count] + CometColumnarExchange #18 + RowToColumnar + WholeStageCodegen (28) HashAggregate [count,count] Project BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -130,8 +204,8 @@ WholeStageCodegen (40) InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (16) + BroadcastExchange #19 + WholeStageCodegen (26) ColumnarToRow InputAdapter CometProject [t_time_sk] @@ -139,13 +213,15 @@ WholeStageCodegen (40) CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (24) - HashAggregate [count] [count(1),h10_30_to_11,count] - InputAdapter - Exchange #15 - WholeStageCodegen (23) + InputAdapter + BroadcastExchange #20 + WholeStageCodegen (34) + ColumnarToRow + InputAdapter + CometHashAggregate [count] + CometColumnarExchange #21 + RowToColumnar + WholeStageCodegen (33) HashAggregate [count,count] Project BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -161,8 +237,8 @@ WholeStageCodegen (40) InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (21) + BroadcastExchange #22 + WholeStageCodegen (31) ColumnarToRow InputAdapter CometProject [t_time_sk] @@ -170,13 +246,15 @@ WholeStageCodegen (40) CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (29) - HashAggregate [count] [count(1),h11_to_11_30,count] - InputAdapter - Exchange #18 - WholeStageCodegen (28) + InputAdapter + BroadcastExchange #23 + WholeStageCodegen (39) + ColumnarToRow + InputAdapter + CometHashAggregate [count] + CometColumnarExchange #24 + RowToColumnar + WholeStageCodegen (38) HashAggregate [count,count] Project BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -192,8 +270,8 @@ WholeStageCodegen (40) InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter - BroadcastExchange #19 - WholeStageCodegen (26) + BroadcastExchange #25 + WholeStageCodegen (36) ColumnarToRow InputAdapter CometProject [t_time_sk] @@ -201,65 +279,3 @@ WholeStageCodegen (40) CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (34) - HashAggregate [count] [count(1),h11_30_to_12,count] - InputAdapter - Exchange #21 - WholeStageCodegen (33) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #22 - WholeStageCodegen (31) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (39) - HashAggregate [count] [count(1),h12_to_12_30,count] - InputAdapter - Exchange #24 - WholeStageCodegen (38) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #25 - WholeStageCodegen (36) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index a59560090d..66c4896f4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -1,31 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (27) -+- * Project (26) - +- * Filter (25) - +- Window (24) - +- * Sort (23) - +- Exchange (22) - +- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.store_sales (4) - : +- ReusedExchange (10) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store (13) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * ColumnarToRow (25) + +- CometSort (24) + +- CometColumnarExchange (23) + +- CometHashAggregate (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.store_sales (4) + : +- ReusedExchange (10) + +- BroadcastExchange (16) + +- * ColumnarToRow (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.store (13) (unknown) Scan parquet spark_catalog.default.item @@ -71,7 +73,7 @@ Join condition: None Output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] Input [8]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] -(10) ReusedExchange [Reuses operator id: 32] +(10) ReusedExchange [Reuses operator id: 34] Output [2]: [d_date_sk#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -119,70 +121,74 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#7))] Aggregate Attributes [1]: [sum#15] Results [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] -(20) Exchange +(20) RowToColumnar Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) HashAggregate [codegen id : 5] +(21) CometColumnarExchange +Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] +Arguments: hashpartitioning(i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) CometHashAggregate Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#7))#17] -Results [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS _w0#19] -(22) Exchange -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(23) CometColumnarExchange +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18] +Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(24) CometSort +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18] +Arguments: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18], [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST] -(23) Sort [codegen id : 6] -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -Arguments: [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST], false, 0 +(25) ColumnarToRow [codegen id : 5] +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18] -(24) Window -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#4, i_brand#2, s_store_name#13, s_company_name#14] +(26) Window +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#19], [i_category#4, i_brand#2, s_store_name#13, s_company_name#14] -(25) Filter [codegen id : 7] -Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] -Condition : CASE WHEN NOT (avg_monthly_sales#20 = 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END +(27) Filter [codegen id : 6] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18, avg_monthly_sales#19] +Condition : CASE WHEN NOT (avg_monthly_sales#19 = 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#19)) / avg_monthly_sales#19) > 0.1000000000000000) END -(26) Project [codegen id : 7] -Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] -Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] +(28) Project [codegen id : 6] +Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, avg_monthly_sales#19] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18, avg_monthly_sales#19] -(27) TakeOrderedAndProject -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] +(29) TakeOrderedAndProject +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, avg_monthly_sales#19] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#19) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, avg_monthly_sales#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (32) -+- * ColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan parquet spark_catalog.default.date_dim (28) +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#21, d_moy#11] +Output [3]: [d_date_sk#10, d_year#20, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(29) CometFilter -Input [3]: [d_date_sk#10, d_year#21, d_moy#11] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 1999)) AND isnotnull(d_date_sk#10)) +(31) CometFilter +Input [3]: [d_date_sk#10, d_year#20, d_moy#11] +Condition : ((isnotnull(d_year#20) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#10)) -(30) CometProject -Input [3]: [d_date_sk#10, d_year#21, d_moy#11] +(32) CometProject +Input [3]: [d_date_sk#10, d_year#20, d_moy#11] Arguments: [d_date_sk#10, d_moy#11], [d_date_sk#10, d_moy#11] -(31) ColumnarToRow [codegen id : 1] +(33) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_moy#11] -(32) BroadcastExchange +(34) BroadcastExchange Input [2]: [d_date_sk#10, d_moy#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index bb9e4e17e2..8007b3089e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -1,17 +1,17 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (7) + WholeStageCodegen (6) Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - Sort [i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometSort [i_category,i_brand,s_store_name,s_company_name] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] + CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + RowToColumnar WholeStageCodegen (4) HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index 1671811424..f277fc9feb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -29,7 +29,7 @@ Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery * Project (12) +- * ColumnarToRow (11) +- CometHashAggregate (10) - +- CometExchange (9) + +- CometColumnarExchange (9) +- CometHashAggregate (8) +- CometProject (7) +- CometFilter (6) @@ -56,9 +56,9 @@ Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] -(9) CometExchange +(9) CometColumnarExchange Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] (10) CometHashAggregate Input [5]: [count#21, sum#22, count#23, sum#24, count#25] @@ -80,7 +80,7 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery * Project (20) +- * ColumnarToRow (19) +- CometHashAggregate (18) - +- CometExchange (17) + +- CometColumnarExchange (17) +- CometHashAggregate (16) +- CometProject (15) +- CometFilter (14) @@ -107,9 +107,9 @@ Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] -(17) CometExchange +(17) CometColumnarExchange Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (18) CometHashAggregate Input [5]: [count#34, sum#35, count#36, sum#37, count#38] @@ -131,7 +131,7 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery * Project (28) +- * ColumnarToRow (27) +- CometHashAggregate (26) - +- CometExchange (25) + +- CometColumnarExchange (25) +- CometHashAggregate (24) +- CometProject (23) +- CometFilter (22) @@ -158,9 +158,9 @@ Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] -(25) CometExchange +(25) CometColumnarExchange Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (26) CometHashAggregate Input [5]: [count#47, sum#48, count#49, sum#50, count#51] @@ -182,7 +182,7 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer * Project (36) +- * ColumnarToRow (35) +- CometHashAggregate (34) - +- CometExchange (33) + +- CometColumnarExchange (33) +- CometHashAggregate (32) +- CometProject (31) +- CometFilter (30) @@ -209,9 +209,9 @@ Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] -(33) CometExchange +(33) CometColumnarExchange Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] (34) CometHashAggregate Input [5]: [count#60, sum#61, count#62, sum#63, count#64] @@ -233,7 +233,7 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer * Project (44) +- * ColumnarToRow (43) +- CometHashAggregate (42) - +- CometExchange (41) + +- CometColumnarExchange (41) +- CometHashAggregate (40) +- CometProject (39) +- CometFilter (38) @@ -260,9 +260,9 @@ Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] -(41) CometExchange +(41) CometColumnarExchange Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] (42) CometHashAggregate Input [5]: [count#73, sum#74, count#75, sum#76, count#77] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index a1fd64ecdc..eb47e062d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometHashAggregate [count,sum,count,sum,count] - CometExchange #1 + CometColumnarExchange #1 CometHashAggregate [ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity] @@ -19,7 +19,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometHashAggregate [count,sum,count,sum,count] - CometExchange #2 + CometColumnarExchange #2 CometHashAggregate [ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity] @@ -32,7 +32,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometHashAggregate [count,sum,count,sum,count] - CometExchange #3 + CometColumnarExchange #3 CometHashAggregate [ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity] @@ -45,7 +45,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometHashAggregate [count,sum,count,sum,count] - CometExchange #4 + CometColumnarExchange #4 CometHashAggregate [ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity] @@ -58,7 +58,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometHashAggregate [count,sum,count,sum,count] - CometExchange #5 + CometColumnarExchange #5 CometHashAggregate [ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt index bb9bf128e7..f75f1c1908 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt @@ -1,55 +1,59 @@ == Physical Plan == -* Project (51) -+- * BroadcastNestedLoopJoin Inner BuildRight (50) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (11) - : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : :- * ColumnarToRow (4) - : : : : +- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- BroadcastExchange (9) - : : : +- * ColumnarToRow (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.household_demographics (5) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan parquet spark_catalog.default.time_dim (12) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.web_page (19) - +- BroadcastExchange (49) - +- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * ColumnarToRow (32) - : : : +- CometProject (31) - : : : +- CometFilter (30) - : : : +- CometScan parquet spark_catalog.default.web_sales (29) - : : +- ReusedExchange (33) - : +- BroadcastExchange (40) - : +- * ColumnarToRow (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.time_dim (36) - +- ReusedExchange (43) +* Project (55) ++- * BroadcastNestedLoopJoin Inner BuildRight (54) + :- * ColumnarToRow (30) + : +- CometHashAggregate (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (11) + : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : :- * ColumnarToRow (4) + : : : : +- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- BroadcastExchange (9) + : : : +- * ColumnarToRow (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.household_demographics (5) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometProject (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.time_dim (12) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometProject (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.web_page (19) + +- BroadcastExchange (53) + +- * ColumnarToRow (52) + +- CometHashAggregate (51) + +- CometColumnarExchange (50) + +- RowToColumnar (49) + +- * HashAggregate (48) + +- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * ColumnarToRow (34) + : : : +- CometProject (33) + : : : +- CometFilter (32) + : : : +- CometScan parquet spark_catalog.default.web_sales (31) + : : +- ReusedExchange (35) + : +- BroadcastExchange (42) + : +- * ColumnarToRow (41) + : +- CometProject (40) + : +- CometFilter (39) + : +- CometScan parquet spark_catalog.default.time_dim (38) + +- ReusedExchange (45) (unknown) Scan parquet spark_catalog.default.web_sales @@ -173,120 +177,128 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#11] Results [1]: [count#12] -(27) Exchange +(27) RowToColumnar Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(28) HashAggregate [codegen id : 10] +(28) CometColumnarExchange +Input [1]: [count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(29) CometHashAggregate Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#13] -Results [1]: [count(1)#13 AS amc#14] + +(30) ColumnarToRow [codegen id : 10] +Input [1]: [amc#13] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] +Output [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] -Condition : ((isnotnull(ws_ship_hdemo_sk#16) AND isnotnull(ws_sold_time_sk#15)) AND isnotnull(ws_web_page_sk#17)) +(32) CometFilter +Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] +Condition : ((isnotnull(ws_ship_hdemo_sk#15) AND isnotnull(ws_sold_time_sk#14)) AND isnotnull(ws_web_page_sk#16)) -(31) CometProject -Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] -Arguments: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17], [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] +(33) CometProject +Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] +Arguments: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16], [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16] -(32) ColumnarToRow [codegen id : 8] -Input [3]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] +(34) ColumnarToRow [codegen id : 8] +Input [3]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16] -(33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#19] +(35) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#18] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#16] -Right keys [1]: [hd_demo_sk#19] +(36) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ws_ship_hdemo_sk#15] +Right keys [1]: [hd_demo_sk#18] Join type: Inner Join condition: None -(35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#15, ws_web_page_sk#17] -Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, hd_demo_sk#19] +(37) Project [codegen id : 8] +Output [2]: [ws_sold_time_sk#14, ws_web_page_sk#16] +Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, hd_demo_sk#18] (unknown) Scan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_hour#21] +Output [2]: [t_time_sk#19, t_hour#20] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct -(37) CometFilter -Input [2]: [t_time_sk#20, t_hour#21] -Condition : (((isnotnull(t_hour#21) AND (t_hour#21 >= 19)) AND (t_hour#21 <= 20)) AND isnotnull(t_time_sk#20)) +(39) CometFilter +Input [2]: [t_time_sk#19, t_hour#20] +Condition : (((isnotnull(t_hour#20) AND (t_hour#20 >= 19)) AND (t_hour#20 <= 20)) AND isnotnull(t_time_sk#19)) -(38) CometProject -Input [2]: [t_time_sk#20, t_hour#21] -Arguments: [t_time_sk#20], [t_time_sk#20] +(40) CometProject +Input [2]: [t_time_sk#19, t_hour#20] +Arguments: [t_time_sk#19], [t_time_sk#19] -(39) ColumnarToRow [codegen id : 6] -Input [1]: [t_time_sk#20] +(41) ColumnarToRow [codegen id : 6] +Input [1]: [t_time_sk#19] -(40) BroadcastExchange -Input [1]: [t_time_sk#20] +(42) BroadcastExchange +Input [1]: [t_time_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#15] -Right keys [1]: [t_time_sk#20] +(43) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ws_sold_time_sk#14] +Right keys [1]: [t_time_sk#19] Join type: Inner Join condition: None -(42) Project [codegen id : 8] -Output [1]: [ws_web_page_sk#17] -Input [3]: [ws_sold_time_sk#15, ws_web_page_sk#17, t_time_sk#20] +(44) Project [codegen id : 8] +Output [1]: [ws_web_page_sk#16] +Input [3]: [ws_sold_time_sk#14, ws_web_page_sk#16, t_time_sk#19] -(43) ReusedExchange [Reuses operator id: 23] -Output [1]: [wp_web_page_sk#22] +(45) ReusedExchange [Reuses operator id: 23] +Output [1]: [wp_web_page_sk#21] -(44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#17] -Right keys [1]: [wp_web_page_sk#22] +(46) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ws_web_page_sk#16] +Right keys [1]: [wp_web_page_sk#21] Join type: Inner Join condition: None -(45) Project [codegen id : 8] +(47) Project [codegen id : 8] Output: [] -Input [2]: [ws_web_page_sk#17, wp_web_page_sk#22] +Input [2]: [ws_web_page_sk#16, wp_web_page_sk#21] -(46) HashAggregate [codegen id : 8] +(48) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#23] -Results [1]: [count#24] +Aggregate Attributes [1]: [count#22] +Results [1]: [count#23] -(47) Exchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(49) RowToColumnar +Input [1]: [count#23] -(48) HashAggregate [codegen id : 9] -Input [1]: [count#24] +(50) CometColumnarExchange +Input [1]: [count#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(51) CometHashAggregate +Input [1]: [count#23] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#25] -Results [1]: [count(1)#25 AS pmc#26] -(49) BroadcastExchange -Input [1]: [pmc#26] +(52) ColumnarToRow [codegen id : 9] +Input [1]: [pmc#24] + +(53) BroadcastExchange +Input [1]: [pmc#24] Arguments: IdentityBroadcastMode, [plan_id=7] -(50) BroadcastNestedLoopJoin [codegen id : 10] +(54) BroadcastNestedLoopJoin [codegen id : 10] Join type: Inner Join condition: None -(51) Project [codegen id : 10] -Output [1]: [(cast(amc#14 as decimal(15,4)) / cast(pmc#26 as decimal(15,4))) AS am_pm_ratio#27] -Input [2]: [amc#14, pmc#26] +(55) Project [codegen id : 10] +Output [1]: [(cast(amc#13 as decimal(15,4)) / cast(pmc#24 as decimal(15,4))) AS am_pm_ratio#25] +Input [2]: [amc#13, pmc#24] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index c4e04b06bc..61f38f3615 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -1,74 +1,78 @@ WholeStageCodegen (10) Project [amc,pmc] BroadcastNestedLoopJoin - HashAggregate [count] [count(1),amc,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometHashAggregate [count] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [count,count] + Project + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_web_page_sk] + BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometProject [t_time_sk] + CometFilter [t_hour,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] + CometProject [wp_web_page_sk] + CometFilter [wp_char_count,wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - HashAggregate [count] [count(1),pmc,count] + ColumnarToRow InputAdapter - Exchange #6 - WholeStageCodegen (8) - HashAggregate [count,count] - Project - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometHashAggregate [count] + CometColumnarExchange #6 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [count,count] + Project + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_web_page_sk] + BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 InputAdapter - ReusedExchange [hd_demo_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - InputAdapter - ReusedExchange [wp_web_page_sk] #4 + ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt index c8110434e8..2d0c9fd987 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt @@ -1,47 +1,49 @@ == Physical Plan == -* Sort (43) -+- Exchange (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * Project (12) - : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.call_center (1) - : : : : : +- BroadcastExchange (7) - : : : : : +- * ColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (4) - : : : : +- ReusedExchange (10) - : : : +- BroadcastExchange (16) - : : : +- * ColumnarToRow (15) - : : : +- CometFilter (14) - : : : +- CometScan parquet spark_catalog.default.customer (13) - : : +- BroadcastExchange (23) - : : +- * ColumnarToRow (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan parquet spark_catalog.default.customer_address (19) - : +- BroadcastExchange (29) - : +- * ColumnarToRow (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.customer_demographics (26) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.household_demographics (32) +* ColumnarToRow (45) ++- CometSort (44) + +- CometColumnarExchange (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (18) + : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : :- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * Project (9) + : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.call_center (1) + : : : : : +- BroadcastExchange (7) + : : : : : +- * ColumnarToRow (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (4) + : : : : +- ReusedExchange (10) + : : : +- BroadcastExchange (16) + : : : +- * ColumnarToRow (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.customer (13) + : : +- BroadcastExchange (23) + : : +- * ColumnarToRow (22) + : : +- CometProject (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.customer_address (19) + : +- BroadcastExchange (29) + : +- * ColumnarToRow (28) + : +- CometFilter (27) + : +- CometScan parquet spark_catalog.default.customer_demographics (26) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.household_demographics (32) (unknown) Scan parquet spark_catalog.default.call_center @@ -87,7 +89,7 @@ Join condition: None Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8] Input [8]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] -(10) ReusedExchange [Reuses operator id: 48] +(10) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#10] (11) BroadcastHashJoin [codegen id : 7] @@ -227,54 +229,58 @@ Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#7))] Aggregate Attributes [1]: [sum#22] Results [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] -(40) Exchange +(40) RowToColumnar Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] -Arguments: hashpartitioning(cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) HashAggregate [codegen id : 8] +(41) CometColumnarExchange +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] +Arguments: hashpartitioning(cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(42) CometHashAggregate Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19] Functions [1]: [sum(UnscaledValue(cr_net_loss#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#7))#24] -Results [4]: [cc_call_center_id#2 AS Call_Center#25, cc_name#3 AS Call_Center_Name#26, cc_manager#4 AS Manager#27, MakeDecimal(sum(UnscaledValue(cr_net_loss#7))#24,17,2) AS Returns_Loss#28] -(42) Exchange -Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] -Arguments: rangepartitioning(Returns_Loss#28 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(43) CometColumnarExchange +Input [4]: [Call_Center#24, Call_Center_Name#25, Manager#26, Returns_Loss#27] +Arguments: rangepartitioning(Returns_Loss#27 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(44) CometSort +Input [4]: [Call_Center#24, Call_Center_Name#25, Manager#26, Returns_Loss#27] +Arguments: [Call_Center#24, Call_Center_Name#25, Manager#26, Returns_Loss#27], [Returns_Loss#27 DESC NULLS LAST] -(43) Sort [codegen id : 9] -Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] -Arguments: [Returns_Loss#28 DESC NULLS LAST], true, 0 +(45) ColumnarToRow [codegen id : 8] +Input [4]: [Call_Center#24, Call_Center_Name#25, Manager#26, Returns_Loss#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#29, d_moy#30] +Output [3]: [d_date_sk#10, d_year#28, d_moy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [3]: [d_date_sk#10, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 1998)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#10)) +(47) CometFilter +Input [3]: [d_date_sk#10, d_year#28, d_moy#29] +Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 1998)) AND (d_moy#29 = 11)) AND isnotnull(d_date_sk#10)) -(46) CometProject -Input [3]: [d_date_sk#10, d_year#29, d_moy#30] +(48) CometProject +Input [3]: [d_date_sk#10, d_year#28, d_moy#29] Arguments: [d_date_sk#10], [d_date_sk#10] -(47) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(48) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index e5d62e3c0b..c2b89020cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -1,11 +1,11 @@ -WholeStageCodegen (9) - Sort [Returns_Loss] +WholeStageCodegen (8) + ColumnarToRow InputAdapter - Exchange [Returns_Loss] #1 - WholeStageCodegen (8) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] - InputAdapter - Exchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 + CometSort [Returns_Loss] + CometColumnarExchange [Returns_Loss] #1 + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] + CometColumnarExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 + RowToColumnar WholeStageCodegen (7) HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index 111c30960b..f4d3879699 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -1,33 +1,37 @@ == Physical Plan == -* HashAggregate (29) -+- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * HashAggregate (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * ColumnarToRow (13) - : : +- CometFilter (12) - : : +- CometScan parquet spark_catalog.default.web_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (24) +* ColumnarToRow (33) ++- CometHashAggregate (32) + +- CometColumnarExchange (31) + +- RowToColumnar (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.item (4) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometHashAggregate (20) + : +- CometColumnarExchange (19) + : +- RowToColumnar (18) + : +- * HashAggregate (17) + : +- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * ColumnarToRow (13) + : : +- CometFilter (12) + : : +- CometScan parquet spark_catalog.default.web_sales (11) + : +- ReusedExchange (14) + +- ReusedExchange (26) (unknown) Scan parquet spark_catalog.default.web_sales @@ -92,7 +96,7 @@ Condition : isnotnull(ws_item_sk#7) (13) ColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -(14) ReusedExchange [Reuses operator id: 34] +(14) ReusedExchange [Reuses operator id: 38] Output [1]: [d_date_sk#11] (15) BroadcastHashJoin [codegen id : 3] @@ -112,96 +116,104 @@ Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] Aggregate Attributes [2]: [sum#12, count#13] Results [3]: [ws_item_sk#7, sum#14, count#15] -(18) Exchange +(18) RowToColumnar Input [3]: [ws_item_sk#7, sum#14, count#15] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(19) HashAggregate [codegen id : 4] +(19) CometColumnarExchange +Input [3]: [ws_item_sk#7, sum#14, count#15] +Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(20) CometHashAggregate Input [3]: [ws_item_sk#7, sum#14, count#15] Keys [1]: [ws_item_sk#7] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#16] -Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] -(20) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) +(21) CometFilter +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) + +(22) ColumnarToRow [codegen id : 4] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -(21) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] +(23) BroadcastExchange +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] Right keys [1]: [ws_item_sk#7] Join type: Inner -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#16) -(23) Project [codegen id : 6] +(25) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -(24) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#18] +(26) ReusedExchange [Reuses operator id: 38] +Output [1]: [d_date_sk#17] -(25) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(28) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#18] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#17] -(27) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#19] -Results [1]: [sum#20] +Aggregate Attributes [1]: [sum#18] +Results [1]: [sum#19] -(28) Exchange -Input [1]: [sum#20] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(30) RowToColumnar +Input [1]: [sum#19] -(29) HashAggregate [codegen id : 7] -Input [1]: [sum#20] +(31) CometColumnarExchange +Input [1]: [sum#19] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(32) CometHashAggregate +Input [1]: [sum#19] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#21] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#21,17,2) AS Excess Discount Amount #22] + +(33) ColumnarToRow [codegen id : 7] +Input [1]: [Excess Discount Amount #20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (34) -+- * ColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.date_dim (30) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#23] +Output [2]: [d_date_sk#17, d_date#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [d_date_sk#18, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) +(35) CometFilter +Input [2]: [d_date_sk#17, d_date#21] +Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 2000-01-27)) AND (d_date#21 <= 2000-04-26)) AND isnotnull(d_date_sk#17)) -(32) CometProject -Input [2]: [d_date_sk#18, d_date#23] -Arguments: [d_date_sk#18], [d_date_sk#18] +(36) CometProject +Input [2]: [d_date_sk#17, d_date#21] +Arguments: [d_date_sk#17], [d_date_sk#17] -(33) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] +(37) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#17] -(34) BroadcastExchange -Input [1]: [d_date_sk#18] +(38) BroadcastExchange +Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index a5e724c1ff..3ef3c57493 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -1,52 +1,56 @@ WholeStageCodegen (7) - HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (6) - HashAggregate [ws_ext_discount_amt] [sum,sum] - Project [ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] - Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_ext_discount_amt] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometHashAggregate [sum] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ws_ext_discount_amt] [sum,sum] + Project [ws_ext_discount_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_discount_amt,ws_sold_date_sk] + BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] + Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_ext_discount_amt] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #4 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometFilter [(1.3 * avg(ws_ext_discount_amt))] + CometHashAggregate [ws_item_sk,sum,count] + CometColumnarExchange [ws_item_sk] #5 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] + Project [ws_item_sk,ws_ext_discount_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] - InputAdapter - Exchange [ws_item_sk] #5 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] - Project [ws_item_sk,ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 4b396dbc38..c7490b740f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -1,27 +1,29 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * ColumnarToRow (12) - : +- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_returns (5) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.reason (13) +* ColumnarToRow (25) ++- CometTakeOrderedAndProject (24) + +- CometHashAggregate (23) + +- CometColumnarExchange (22) + +- RowToColumnar (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * ColumnarToRow (12) + : +- CometProject (11) + : +- CometSortMergeJoin (10) + : :- CometSort (4) + : : +- CometColumnarExchange (3) + : : +- CometProject (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometSort (9) + : +- CometColumnarExchange (8) + : +- CometProject (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_returns (5) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometProject (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.reason (13) (unknown) Scan parquet spark_catalog.default.store_sales @@ -34,9 +36,9 @@ ReadSchema: struct Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] -(9) CometExchange +(9) CometColumnarExchange Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (10) CometSort Input [2]: [ws_warehouse_sk#9, ws_order_number#10] @@ -113,9 +115,9 @@ ReadSchema: struct Input [2]: [wr_order_number#12, wr_returned_date_sk#13] Arguments: [wr_order_number#12], [wr_order_number#12] -(16) CometExchange +(16) CometColumnarExchange Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (17) CometSort Input [1]: [wr_order_number#12] @@ -247,14 +249,18 @@ Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(Unscaled Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21, count(ws_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) Exchange +(44) RowToColumnar Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 9] +(45) CometColumnarExchange +Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(46) CometHashAggregate Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21, count(ws_order_number#5)#24] -Results [3]: [count(ws_order_number#5)#24 AS order count #26, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#20,17,2) AS total shipping cost #27, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#21,17,2) AS total net profit #28] + +(47) ColumnarToRow [codegen id : 9] +Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 369065a668..1784692e7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -1,68 +1,70 @@ WholeStageCodegen (9) - HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (8) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (3) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometSort [ws_order_number] - CometExchange [ws_order_number] #2 - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometSort [ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + CometHashAggregate [sum,sum,count] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (3) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [ws_order_number] + CometColumnarExchange [ws_order_number] #2 + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [ws_order_number] + CometColumnarExchange [ws_order_number] #3 + CometProject [ws_warehouse_sk,ws_order_number] + CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [wr_order_number] + CometColumnarExchange [wr_order_number] #4 + CometProject [wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometProject [web_site_sk] + CometFilter [web_company_name,web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 375c7bf2f9..c9e31178a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -1,65 +1,67 @@ == Physical Plan == -* HashAggregate (61) -+- Exchange (60) - +- * HashAggregate (59) - +- * HashAggregate (58) - +- * HashAggregate (57) - +- * Project (56) - +- * BroadcastHashJoin Inner BuildRight (55) - :- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * SortMergeJoin LeftSemi (35) - : : : :- * SortMergeJoin LeftSemi (18) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- * Project (17) - : : : : +- * SortMergeJoin Inner (16) - : : : : :- * ColumnarToRow (12) - : : : : : +- CometSort (11) - : : : : : +- CometExchange (10) - : : : : : +- CometProject (9) - : : : : : +- CometFilter (8) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) - : : : : +- * ColumnarToRow (15) - : : : : +- CometSort (14) - : : : : +- ReusedExchange (13) - : : : +- * Project (34) - : : : +- * SortMergeJoin Inner (33) - : : : :- * ColumnarToRow (24) - : : : : +- CometSort (23) - : : : : +- CometExchange (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.web_returns (19) - : : : +- * Project (32) - : : : +- * SortMergeJoin Inner (31) - : : : :- * ColumnarToRow (27) - : : : : +- CometSort (26) - : : : : +- ReusedExchange (25) - : : : +- * ColumnarToRow (30) - : : : +- CometSort (29) - : : : +- ReusedExchange (28) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.date_dim (36) - : +- BroadcastExchange (47) - : +- * ColumnarToRow (46) - : +- CometProject (45) - : +- CometFilter (44) - : +- CometScan parquet spark_catalog.default.customer_address (43) - +- BroadcastExchange (54) - +- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.web_site (50) +* ColumnarToRow (63) ++- CometHashAggregate (62) + +- CometColumnarExchange (61) + +- RowToColumnar (60) + +- * HashAggregate (59) + +- * HashAggregate (58) + +- * HashAggregate (57) + +- * Project (56) + +- * BroadcastHashJoin Inner BuildRight (55) + :- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * SortMergeJoin LeftSemi (35) + : : : :- * SortMergeJoin LeftSemi (18) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- * Project (17) + : : : : +- * SortMergeJoin Inner (16) + : : : : :- * ColumnarToRow (12) + : : : : : +- CometSort (11) + : : : : : +- CometColumnarExchange (10) + : : : : : +- CometProject (9) + : : : : : +- CometFilter (8) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) + : : : : +- * ColumnarToRow (15) + : : : : +- CometSort (14) + : : : : +- ReusedExchange (13) + : : : +- * Project (34) + : : : +- * SortMergeJoin Inner (33) + : : : :- * ColumnarToRow (24) + : : : : +- CometSort (23) + : : : : +- CometColumnarExchange (22) + : : : : +- CometProject (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.web_returns (19) + : : : +- * Project (32) + : : : +- * SortMergeJoin Inner (31) + : : : :- * ColumnarToRow (27) + : : : : +- CometSort (26) + : : : : +- ReusedExchange (25) + : : : +- * ColumnarToRow (30) + : : : +- CometSort (29) + : : : +- ReusedExchange (28) + : : +- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.date_dim (36) + : +- BroadcastExchange (47) + : +- * ColumnarToRow (46) + : +- CometProject (45) + : +- CometFilter (44) + : +- CometScan parquet spark_catalog.default.customer_address (43) + +- BroadcastExchange (54) + +- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.web_site (50) (unknown) Scan parquet spark_catalog.default.web_sales @@ -77,9 +79,9 @@ Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -(4) CometExchange +(4) CometColumnarExchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] (5) CometSort Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] @@ -103,9 +105,9 @@ Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] -(10) CometExchange +(10) CometColumnarExchange Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (11) CometSort Input [2]: [ws_warehouse_sk#8, ws_order_number#9] @@ -155,9 +157,9 @@ Condition : isnotnull(wr_order_number#13) Input [2]: [wr_order_number#13, wr_returned_date_sk#14] Arguments: [wr_order_number#13], [wr_order_number#13] -(22) CometExchange +(22) CometColumnarExchange Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (23) CometSort Input [1]: [wr_order_number#13] @@ -329,14 +331,18 @@ Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(Unscaled Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22, count(ws_order_number#4)#25] Results [3]: [sum#23, sum#24, count#26] -(60) Exchange +(60) RowToColumnar Input [3]: [sum#23, sum#24, count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(61) HashAggregate [codegen id : 15] +(61) CometColumnarExchange +Input [3]: [sum#23, sum#24, count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(62) CometHashAggregate Input [3]: [sum#23, sum#24, count#26] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22, count(ws_order_number#4)#25] -Results [3]: [count(ws_order_number#4)#25 AS order count #27, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#21,17,2) AS total shipping cost #28, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#22,17,2) AS total net profit #29] + +(63) ColumnarToRow [codegen id : 15] +Input [3]: [order count #27, total shipping cost #28, total net profit #29] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 2ad651cb64..335ef0af24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -1,99 +1,101 @@ WholeStageCodegen (15) - HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (14) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (5) - SortMergeJoin [ws_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (1) - ColumnarToRow + CometHashAggregate [sum,sum,count] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (5) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometSort [ws_order_number] + CometColumnarExchange [ws_order_number] #2 + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometSort [ws_order_number] + CometColumnarExchange [ws_order_number] #3 + CometProject [ws_warehouse_sk,ws_order_number] + CometFilter [ws_order_number,ws_warehouse_sk] + CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometSort [ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (10) + Project [wr_order_number] + SortMergeJoin [wr_order_number,ws_order_number] InputAdapter - CometSort [ws_order_number] - CometExchange [ws_order_number] #2 - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - ColumnarToRow + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometSort [wr_order_number] + CometColumnarExchange [wr_order_number] #4 + CometProject [wr_order_number] + CometFilter [wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + WholeStageCodegen (9) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] InputAdapter - CometSort [ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] - CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (3) - ColumnarToRow + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometSort [ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter - CometSort [ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (10) - Project [wr_order_number] - SortMergeJoin [wr_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number] - CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - InputAdapter - WholeStageCodegen (9) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometSort [ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometSort [ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometSort [ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) + BroadcastExchange #6 + WholeStageCodegen (12) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (12) + BroadcastExchange #7 + WholeStageCodegen (13) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (13) - ColumnarToRow - InputAdapter - CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometProject [web_site_sk] + CometFilter [web_company_name,web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt index b55971ac80..49062fc63a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt @@ -1,32 +1,34 @@ == Physical Plan == -* HashAggregate (28) -+- Exchange (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.household_demographics (5) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometProject (14) - : +- CometFilter (13) - : +- CometScan parquet spark_catalog.default.time_dim (12) - +- BroadcastExchange (23) - +- * ColumnarToRow (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan parquet spark_catalog.default.store (19) +* ColumnarToRow (30) ++- CometHashAggregate (29) + +- CometColumnarExchange (28) + +- RowToColumnar (27) + +- * HashAggregate (26) + +- * Project (25) + +- * BroadcastHashJoin Inner BuildRight (24) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- BroadcastExchange (9) + : : +- * ColumnarToRow (8) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.household_demographics (5) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometProject (14) + : +- CometFilter (13) + : +- CometScan parquet spark_catalog.default.time_dim (12) + +- BroadcastExchange (23) + +- * ColumnarToRow (22) + +- CometProject (21) + +- CometFilter (20) + +- CometScan parquet spark_catalog.default.store (19) (unknown) Scan parquet spark_catalog.default.store_sales @@ -150,14 +152,18 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#12] Results [1]: [count#13] -(27) Exchange +(27) RowToColumnar Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(28) HashAggregate [codegen id : 5] +(28) CometColumnarExchange +Input [1]: [count#13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(29) CometHashAggregate Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#14] -Results [1]: [count(1)#14 AS count(1)#15] + +(30) ColumnarToRow [codegen id : 5] +Input [1]: [count(1)#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt index d1438f48eb..647015f01c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt @@ -1,41 +1,43 @@ WholeStageCodegen (5) - HashAggregate [count] [count(1),count(1),count] + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometHashAggregate [count] + CometColumnarExchange #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometProject [s_store_sk] + CometFilter [s_store_name,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt index 512037f6a3..7f3d3ee195 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt @@ -1,27 +1,30 @@ == Physical Plan == -* HashAggregate (23) -+- Exchange (22) - +- * HashAggregate (21) - +- * Project (20) - +- * SortMergeJoin FullOuter (19) - :- * Sort (9) - : +- * HashAggregate (8) - : +- Exchange (7) - : +- * HashAggregate (6) - : +- * Project (5) - : +- * BroadcastHashJoin Inner BuildRight (4) - : :- * ColumnarToRow (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (3) - +- * Sort (18) - +- * HashAggregate (17) - +- Exchange (16) - +- * HashAggregate (15) - +- * Project (14) - +- * BroadcastHashJoin Inner BuildRight (13) - :- * ColumnarToRow (11) - : +- CometScan parquet spark_catalog.default.catalog_sales (10) - +- ReusedExchange (12) +* ColumnarToRow (26) ++- CometHashAggregate (25) + +- CometColumnarExchange (24) + +- CometHashAggregate (23) + +- CometProject (22) + +- CometSortMergeJoin (21) + :- CometSort (10) + : +- CometHashAggregate (9) + : +- CometColumnarExchange (8) + : +- RowToColumnar (7) + : +- * HashAggregate (6) + : +- * Project (5) + : +- * BroadcastHashJoin Inner BuildRight (4) + : :- * ColumnarToRow (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- ReusedExchange (3) + +- CometSort (20) + +- CometHashAggregate (19) + +- CometColumnarExchange (18) + +- RowToColumnar (17) + +- * HashAggregate (16) + +- * Project (15) + +- * BroadcastHashJoin Inner BuildRight (14) + :- * ColumnarToRow (12) + : +- CometScan parquet spark_catalog.default.catalog_sales (11) + +- ReusedExchange (13) (unknown) Scan parquet spark_catalog.default.store_sales @@ -34,7 +37,7 @@ ReadSchema: struct (2) ColumnarToRow [codegen id : 2] Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -(3) ReusedExchange [Reuses operator id: 28] +(3) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#5] (4) BroadcastHashJoin [codegen id : 2] @@ -54,20 +57,21 @@ Functions: [] Aggregate Attributes: [] Results [2]: [ss_customer_sk#2, ss_item_sk#1] -(7) Exchange +(7) RowToColumnar Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(8) HashAggregate [codegen id : 3] +(8) CometColumnarExchange +Input [2]: [ss_customer_sk#2, ss_item_sk#1] +Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(9) CometHashAggregate Input [2]: [ss_customer_sk#2, ss_item_sk#1] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] -Aggregate Attributes: [] -Results [2]: [ss_customer_sk#2 AS customer_sk#6, ss_item_sk#1 AS item_sk#7] -(9) Sort [codegen id : 3] +(10) CometSort Input [2]: [customer_sk#6, item_sk#7] -Arguments: [customer_sk#6 ASC NULLS FIRST, item_sk#7 ASC NULLS FIRST], false, 0 +Arguments: [customer_sk#6, item_sk#7], [customer_sk#6 ASC NULLS FIRST, item_sk#7 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] @@ -76,104 +80,103 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#11)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 5] +(12) ColumnarToRow [codegen id : 4] Input [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] -(12) ReusedExchange [Reuses operator id: 28] +(13) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#12] -(13) BroadcastHashJoin [codegen id : 5] +(14) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(14) Project [codegen id : 5] +(15) Project [codegen id : 4] Output [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Input [4]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10, d_date_sk#12] -(15) HashAggregate [codegen id : 5] +(16) HashAggregate [codegen id : 4] Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Functions: [] Aggregate Attributes: [] Results [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -(16) Exchange +(17) RowToColumnar +Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] + +(18) CometColumnarExchange Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Arguments: hashpartitioning(cs_bill_customer_sk#8, cs_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(cs_bill_customer_sk#8, cs_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) HashAggregate [codegen id : 6] +(19) CometHashAggregate Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Functions: [] -Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#8 AS customer_sk#13, cs_item_sk#9 AS item_sk#14] -(18) Sort [codegen id : 6] +(20) CometSort Input [2]: [customer_sk#13, item_sk#14] -Arguments: [customer_sk#13 ASC NULLS FIRST, item_sk#14 ASC NULLS FIRST], false, 0 +Arguments: [customer_sk#13, item_sk#14], [customer_sk#13 ASC NULLS FIRST, item_sk#14 ASC NULLS FIRST] -(19) SortMergeJoin [codegen id : 7] -Left keys [2]: [customer_sk#6, item_sk#7] -Right keys [2]: [customer_sk#13, item_sk#14] -Join type: FullOuter -Join condition: None +(21) CometSortMergeJoin +Left output [2]: [customer_sk#6, item_sk#7] +Right output [2]: [customer_sk#13, item_sk#14] +Arguments: [customer_sk#6, item_sk#7], [customer_sk#13, item_sk#14], FullOuter -(20) Project [codegen id : 7] -Output [2]: [customer_sk#6, customer_sk#13] +(22) CometProject Input [4]: [customer_sk#6, item_sk#7, customer_sk#13, item_sk#14] +Arguments: [customer_sk#6, customer_sk#13], [customer_sk#6, customer_sk#13] -(21) HashAggregate [codegen id : 7] +(23) CometHashAggregate Input [2]: [customer_sk#6, customer_sk#13] Keys: [] Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#15, sum#16, sum#17] -Results [3]: [sum#18, sum#19, sum#20] -(22) Exchange -Input [3]: [sum#18, sum#19, sum#20] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +(24) CometColumnarExchange +Input [3]: [sum#15, sum#16, sum#17] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) HashAggregate [codegen id : 8] -Input [3]: [sum#18, sum#19, sum#20] +(25) CometHashAggregate +Input [3]: [sum#15, sum#16, sum#17] Keys: [] Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END)#21, sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#22, sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#23] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END)#21 AS store_only#24, sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#22 AS catalog_only#25, sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#23 AS store_and_catalog#26] + +(26) ColumnarToRow [codegen id : 5] +Input [3]: [store_only#18, catalog_only#19, store_and_catalog#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * ColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.date_dim (24) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#27] +Output [2]: [d_date_sk#5, d_month_seq#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#27] -Condition : (((isnotnull(d_month_seq#27) AND (d_month_seq#27 >= 1200)) AND (d_month_seq#27 <= 1211)) AND isnotnull(d_date_sk#5)) +(28) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#21] +Condition : (((isnotnull(d_month_seq#21) AND (d_month_seq#21 >= 1200)) AND (d_month_seq#21 <= 1211)) AND isnotnull(d_date_sk#5)) -(26) CometProject -Input [2]: [d_date_sk#5, d_month_seq#27] +(29) CometProject +Input [2]: [d_date_sk#5, d_month_seq#21] Arguments: [d_date_sk#5], [d_date_sk#5] -(27) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(28) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt index be9c20a560..f4381c85e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt @@ -1,47 +1,43 @@ -WholeStageCodegen (8) - HashAggregate [sum,sum,sum] [sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),store_only,catalog_only,store_and_catalog,sum,sum,sum] +WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange #1 - WholeStageCodegen (7) - HashAggregate [customer_sk,customer_sk] [sum,sum,sum,sum,sum,sum] - Project [customer_sk,customer_sk] - SortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [customer_sk,item_sk] - HashAggregate [ss_customer_sk,ss_item_sk] [customer_sk,item_sk] - InputAdapter - Exchange [ss_customer_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - WholeStageCodegen (6) - Sort [customer_sk,item_sk] - HashAggregate [cs_bill_customer_sk,cs_item_sk] [customer_sk,item_sk] - InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #4 - WholeStageCodegen (5) - HashAggregate [cs_bill_customer_sk,cs_item_sk] - Project [cs_bill_customer_sk,cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometHashAggregate [sum,sum,sum] + CometColumnarExchange #1 + CometHashAggregate [customer_sk,customer_sk] + CometProject [customer_sk,customer_sk] + CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] + CometSort [customer_sk,item_sk] + CometHashAggregate [ss_customer_sk,ss_item_sk] + CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ss_customer_sk,ss_item_sk] + Project [ss_item_sk,ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 + CometSort [customer_sk,item_sk] + CometHashAggregate [cs_bill_customer_sk,cs_item_sk] + CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #4 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [cs_bill_customer_sk,cs_item_sk] + Project [cs_bill_customer_sk,cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index b69f690810..457e702c53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -1,26 +1,30 @@ == Physical Plan == -* Project (22) -+- * Sort (21) - +- Exchange (20) - +- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +* ColumnarToRow (26) ++- CometProject (25) + +- CometSort (24) + +- CometColumnarExchange (23) + +- RowToColumnar (22) + +- * Project (21) + +- Window (20) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.store_sales @@ -66,7 +70,7 @@ Join condition: None Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 27] +(10) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -86,74 +90,84 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(14) RowToColumnar Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(15) CometColumnarExchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] -Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] -(16) Exchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(17) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) Sort [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [i_class#9 ASC NULLS FIRST], false, 0 +(18) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(18) Window -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(19) ColumnarToRow [codegen id : 4] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] -(19) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] -Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] +(20) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(20) Exchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(21) Project [codegen id : 5] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] -(21) Sort [codegen id : 7] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], true, 0 +(22) RowToColumnar +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] -(22) Project [codegen id : 7] -Output [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +(23) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(24) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6], [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST] + +(25) CometProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] + +(26) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +Output [2]: [d_date_sk#11, d_date#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(28) CometFilter +Input [2]: [d_date_sk#11, d_date#18] +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(25) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(29) CometProject +Input [2]: [d_date_sk#11, d_date#18] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index 9eabb9977c..2e9931a8ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -1,44 +1,46 @@ -WholeStageCodegen (7) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] - InputAdapter - Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - Sort [i_class] - InputAdapter - Exchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometSort [i_category,i_class,i_item_id,i_item_desc,revenueratio] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + RowToColumnar + WholeStageCodegen (5) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [i_class] + CometColumnarExchange [i_class] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt index 8420e644cc..c8d7d7eaf3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt @@ -1,36 +1,38 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.ship_mode (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.call_center (16) - +- BroadcastExchange (26) - +- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +* ColumnarToRow (34) ++- CometTakeOrderedAndProject (33) + +- CometHashAggregate (32) + +- CometColumnarExchange (31) + +- RowToColumnar (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.warehouse (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.ship_mode (10) + : +- BroadcastExchange (19) + : +- * ColumnarToRow (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.call_center (16) + +- BroadcastExchange (26) + +- * ColumnarToRow (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -170,18 +172,22 @@ Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) < Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] Results [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -(30) Exchange +(30) RowToColumnar Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(31) HashAggregate [codegen id : 6] +(31) CometColumnarExchange +Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(32) CometHashAggregate Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [3]: [_groupingexpression#14, sm_type#9, cc_name#11] Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] -Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] -(32) TakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] -Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] +(33) CometTakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#25, sm_type#9, cc_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#25 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,cc_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#25,sm_type#9,cc_name#11,30 days #26,31 - 60 days #27,61 - 90 days #28,91 - 120 days #29,>120 days #30]), 100, [substr(w_warehouse_name, 1, 20)#25 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#25, sm_type#9, cc_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] + +(34) ColumnarToRow [codegen id : 6] +Input [8]: [substr(w_warehouse_name, 1, 20)#25, sm_type#9, cc_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt index c5f25f0795..e24461c48c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt @@ -1,48 +1,50 @@ -TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) - HashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - InputAdapter - Exchange [_groupingexpression,sm_type,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,w_warehouse_name] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] +WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] + CometColumnarExchange [_groupingexpression,sm_type,cc_name] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,w_warehouse_name] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + Project [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometFilter [sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometFilter [cc_call_center_sk] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cc_call_center_sk] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index ea5dac96e7..b413027c7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -1,45 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * ColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (35) - +- * ColumnarToRow (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.customer_demographics (32) +* ColumnarToRow (43) ++- CometTakeOrderedAndProject (42) + +- CometHashAggregate (41) + +- CometColumnarExchange (40) + +- RowToColumnar (39) + +- * HashAggregate (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : +- BroadcastExchange (9) + : : : +- * Project (8) + : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : :- * ColumnarToRow (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (6) + : : +- BroadcastExchange (22) + : : +- Union (21) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * ColumnarToRow (17) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (18) + : +- BroadcastExchange (29) + : +- * ColumnarToRow (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.customer_address (25) + +- BroadcastExchange (35) + +- * ColumnarToRow (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.customer_demographics (32) (unknown) Scan parquet spark_catalog.default.customer @@ -66,7 +68,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) ReusedExchange [Reuses operator id: 46] +(6) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#7] (7) BroadcastHashJoin [codegen id : 2] @@ -99,7 +101,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -(13) ReusedExchange [Reuses operator id: 46] +(13) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#11] (14) BroadcastHashJoin [codegen id : 4] @@ -122,7 +124,7 @@ ReadSchema: struct (17) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(18) ReusedExchange [Reuses operator id: 46] +(18) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#16] (19) BroadcastHashJoin [codegen id : 6] @@ -218,50 +220,54 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -(39) Exchange +(39) RowToColumnar Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(40) HashAggregate [codegen id : 10] +(40) CometColumnarExchange +Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] +Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(41) CometHashAggregate Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] -(41) TakeOrderedAndProject -Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] -Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] +(42) CometTakeOrderedAndProject +Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#21 ASC NULLS FIRST,cd_marital_status#22 ASC NULLS FIRST,cd_education_status#23 ASC NULLS FIRST,cd_purchase_estimate#24 ASC NULLS FIRST,cd_credit_rating#25 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[cd_gender#21,cd_marital_status#22,cd_education_status#23,cnt1#31,cd_purchase_estimate#24,cnt2#32,cd_credit_rating#25,cnt3#33,cd_dep_count#26,cnt4#34,cd_dep_employed_count#27,cnt5#35,cd_dep_college_count#28,cnt6#36]), 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] + +(43) ColumnarToRow [codegen id : 10] +Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (46) -+- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.date_dim (42) +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#38, d_moy#39] +Output [3]: [d_date_sk#7, d_year#37, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter -Input [3]: [d_date_sk#7, d_year#38, d_moy#39] -Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 4)) AND (d_moy#39 <= 7)) AND isnotnull(d_date_sk#7)) +(45) CometFilter +Input [3]: [d_date_sk#7, d_year#37, d_moy#38] +Condition : (((((isnotnull(d_year#37) AND isnotnull(d_moy#38)) AND (d_year#37 = 2002)) AND (d_moy#38 >= 4)) AND (d_moy#38 <= 7)) AND isnotnull(d_date_sk#7)) -(44) CometProject -Input [3]: [d_date_sk#7, d_year#38, d_moy#39] +(46) CometProject +Input [3]: [d_date_sk#7, d_year#37, d_moy#38] Arguments: [d_date_sk#7], [d_date_sk#7] -(45) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(46) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 3eb2210a6e..440bda63b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -1,72 +1,74 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - InputAdapter - Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow +WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] + CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + Union + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #5 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_county,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index befc87707c..0052afc39e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -1,75 +1,83 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * ColumnarToRow (36) - : : : +- CometFilter (35) - : : : +- CometScan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) - +- BroadcastExchange (68) - +- * HashAggregate (67) - +- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * ColumnarToRow (55) - : : +- CometFilter (54) - : : +- CometScan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) +TakeOrderedAndProject (79) ++- * Project (78) + +- * BroadcastHashJoin Inner BuildRight (77) + :- * Project (58) + : +- * BroadcastHashJoin Inner BuildRight (57) + : :- * BroadcastHashJoin Inner BuildRight (37) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometColumnarExchange (15) + : : : +- RowToColumnar (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (36) + : : +- * ColumnarToRow (35) + : : +- CometHashAggregate (34) + : : +- CometColumnarExchange (33) + : : +- RowToColumnar (32) + : : +- * HashAggregate (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * Project (27) + : : : +- * BroadcastHashJoin Inner BuildRight (26) + : : : :- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.store_sales (22) + : : +- ReusedExchange (28) + : +- BroadcastExchange (56) + : +- * ColumnarToRow (55) + : +- CometFilter (54) + : +- CometHashAggregate (53) + : +- CometColumnarExchange (52) + : +- RowToColumnar (51) + : +- * HashAggregate (50) + : +- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (46) + : : +- * BroadcastHashJoin Inner BuildRight (45) + : : :- * ColumnarToRow (40) + : : : +- CometFilter (39) + : : : +- CometScan parquet spark_catalog.default.customer (38) + : : +- BroadcastExchange (44) + : : +- * ColumnarToRow (43) + : : +- CometFilter (42) + : : +- CometScan parquet spark_catalog.default.web_sales (41) + : +- ReusedExchange (47) + +- BroadcastExchange (76) + +- * ColumnarToRow (75) + +- CometHashAggregate (74) + +- CometColumnarExchange (73) + +- RowToColumnar (72) + +- * HashAggregate (71) + +- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * ColumnarToRow (61) + : : +- CometFilter (60) + : : +- CometScan parquet spark_catalog.default.customer (59) + : +- BroadcastExchange (65) + : +- * ColumnarToRow (64) + : +- CometFilter (63) + : +- CometScan parquet spark_catalog.default.web_sales (62) + +- ReusedExchange (68) (unknown) Scan parquet spark_catalog.default.customer @@ -115,7 +123,7 @@ Join condition: None Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(10) ReusedExchange [Reuses operator id: 75] +(10) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#14, d_year#15] (11) BroadcastHashJoin [codegen id : 3] @@ -135,296 +143,312 @@ Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discoun Aggregate Attributes [1]: [sum#16] Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -(14) Exchange +(14) RowToColumnar Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 16] +(15) CometColumnarExchange +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] -Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] -(16) Filter [codegen id : 16] -Input [2]: [customer_id#19, year_total#20] -Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) +(17) CometFilter +Input [2]: [customer_id#18, year_total#19] +Condition : (isnotnull(year_total#19) AND (year_total#19 > 0.00)) + +(18) ColumnarToRow [codegen id : 16] +Input [2]: [customer_id#18, year_total#19] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Output [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter -Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) +(20) CometFilter +Input [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_customer_id#21)) -(19) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +(21) ColumnarToRow [codegen id : 6] +Input [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#33)] +PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -Condition : isnotnull(ss_customer_sk#29) +(23) CometFilter +Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Condition : isnotnull(ss_customer_sk#28) -(22) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +(24) ColumnarToRow [codegen id : 4] +Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -(23) BroadcastExchange -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +(25) BroadcastExchange +Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#21] -Right keys [1]: [ss_customer_sk#29] +(26) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#20] +Right keys [1]: [ss_customer_sk#28] Join type: Inner Join condition: None -(25) Project [codegen id : 6] -Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +(27) Project [codegen id : 6] +Output [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Input [12]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -(26) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#34, d_year#35] +(28) ReusedExchange [Reuses operator id: 87] +Output [2]: [d_date_sk#33, d_year#34] -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#34] +(29) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#31] +Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] -Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] - -(29) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] -Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] -Aggregate Attributes [1]: [sum#36] -Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] - -(30) Exchange -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] -Results [5]: [c_customer_id#22 AS customer_id#38, c_first_name#23 AS customer_first_name#39, c_last_name#24 AS customer_last_name#40, c_email_address#28 AS customer_email_address#41, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#42] - -(32) BroadcastExchange -Input [5]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42] +(30) Project [codegen id : 6] +Output [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] +Input [12]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31, d_date_sk#33, d_year#34] + +(31) HashAggregate [codegen id : 6] +Input [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] +Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] +Aggregate Attributes [1]: [sum#35] +Results [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] + +(32) RowToColumnar +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] + +(33) CometColumnarExchange +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] +Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometHashAggregate +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] +Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] + +(35) ColumnarToRow [codegen id : 7] +Input [5]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40, year_total#41] + +(36) BroadcastExchange +Input [5]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40, year_total#41] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#38] +(37) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#37] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] +Output [8]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter -Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] -Condition : (isnotnull(c_customer_sk#43) AND isnotnull(c_customer_id#44)) +(39) CometFilter +Input [8]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49] +Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_customer_id#43)) -(36) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] +(40) ColumnarToRow [codegen id : 10] +Input [8]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Output [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#51) +(42) CometFilter +Input [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_bill_customer_sk#50) -(39) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +(43) ColumnarToRow [codegen id : 8] +Input [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] -(40) BroadcastExchange -Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +(44) BroadcastExchange +Input [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#43] -Right keys [1]: [ws_bill_customer_sk#51] +(45) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#42] +Right keys [1]: [ws_bill_customer_sk#50] Join type: Inner Join condition: None -(42) Project [codegen id : 10] -Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -Input [12]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +(46) Project [codegen id : 10] +Output [10]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] +Input [12]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] -(43) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#56, d_year#57] +(47) ReusedExchange [Reuses operator id: 83] +Output [2]: [d_date_sk#55, d_year#56] -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#54] -Right keys [1]: [d_date_sk#56] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#53] +Right keys [1]: [d_date_sk#55] Join type: Inner Join condition: None -(45) Project [codegen id : 10] -Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] -Input [12]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] - -(46) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] -Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] -Aggregate Attributes [1]: [sum#58] -Results [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] - -(47) Exchange -Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] -Arguments: hashpartitioning(c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(48) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] -Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60] -Results [2]: [c_customer_id#44 AS customer_id#61, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60,18,2) AS year_total#62] - -(49) Filter [codegen id : 11] -Input [2]: [customer_id#61, year_total#62] -Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.00)) - -(50) BroadcastExchange -Input [2]: [customer_id#61, year_total#62] +(49) Project [codegen id : 10] +Output [10]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, ws_ext_discount_amt#51, ws_ext_list_price#52, d_year#56] +Input [12]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] + +(50) HashAggregate [codegen id : 10] +Input [10]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, ws_ext_discount_amt#51, ws_ext_list_price#52, d_year#56] +Keys [8]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#52 - ws_ext_discount_amt#51)))] +Aggregate Attributes [1]: [sum#57] +Results [9]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56, sum#58] + +(51) RowToColumnar +Input [9]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56, sum#58] + +(52) CometColumnarExchange +Input [9]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56, sum#58] +Arguments: hashpartitioning(c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(53) CometHashAggregate +Input [9]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56, sum#58] +Keys [8]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#52 - ws_ext_discount_amt#51)))] + +(54) CometFilter +Input [2]: [customer_id#59, year_total#60] +Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.00)) + +(55) ColumnarToRow [codegen id : 11] +Input [2]: [customer_id#59, year_total#60] + +(56) BroadcastExchange +Input [2]: [customer_id#59, year_total#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#61] +(57) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#59] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [8]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62] -Input [9]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, customer_id#61, year_total#62] +(58) Project [codegen id : 16] +Output [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40, year_total#41, year_total#60] +Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40, year_total#41, customer_id#59, year_total#60] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] +Output [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter -Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] -Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_customer_id#64)) +(60) CometFilter +Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] +Condition : (isnotnull(c_customer_sk#61) AND isnotnull(c_customer_id#62)) -(55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] +(61) ColumnarToRow [codegen id : 14] +Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +Output [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] 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#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] -Condition : isnotnull(ws_bill_customer_sk#71) +(63) CometFilter +Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Condition : isnotnull(ws_bill_customer_sk#69) -(58) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +(64) ColumnarToRow [codegen id : 12] +Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -(59) BroadcastExchange -Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +(65) BroadcastExchange +Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#63] -Right keys [1]: [ws_bill_customer_sk#71] +(66) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#61] +Right keys [1]: [ws_bill_customer_sk#69] Join type: Inner Join condition: None -(61) Project [codegen id : 14] -Output [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] -Input [12]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +(67) Project [codegen id : 14] +Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Input [12]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -(62) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#76, d_year#77] +(68) ReusedExchange [Reuses operator id: 87] +Output [2]: [d_date_sk#74, d_year#75] -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#74] -Right keys [1]: [d_date_sk#76] +(69) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#72] +Right keys [1]: [d_date_sk#74] Join type: Inner Join condition: None -(64) Project [codegen id : 14] -Output [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] -Input [12]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74, d_date_sk#76, d_year#77] - -(65) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] -Keys [8]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))] -Aggregate Attributes [1]: [sum#78] -Results [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] - -(66) Exchange -Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] -Arguments: hashpartitioning(c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(67) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] -Keys [8]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))#60] -Results [2]: [c_customer_id#64 AS customer_id#80, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))#60,18,2) AS year_total#81] - -(68) BroadcastExchange -Input [2]: [customer_id#80, year_total#81] +(70) Project [codegen id : 14] +Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] +Input [12]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72, d_date_sk#74, d_year#75] + +(71) HashAggregate [codegen id : 14] +Input [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] +Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] +Aggregate Attributes [1]: [sum#76] +Results [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] + +(72) RowToColumnar +Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] + +(73) CometColumnarExchange +Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] +Arguments: hashpartitioning(c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(74) CometHashAggregate +Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] +Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] + +(75) ColumnarToRow [codegen id : 15] +Input [2]: [customer_id#78, year_total#79] + +(76) BroadcastExchange +Input [2]: [customer_id#78, year_total#79] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(69) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#80] +(77) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#78] Join type: Inner -Join condition: (CASE WHEN (year_total#62 > 0.00) THEN (year_total#81 / year_total#62) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END) +Join condition: (CASE WHEN (year_total#60 > 0.00) THEN (year_total#79 / year_total#60) ELSE 0E-20 END > CASE WHEN (year_total#19 > 0.00) THEN (year_total#41 / year_total#19) ELSE 0E-20 END) -(70) Project [codegen id : 16] -Output [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] -Input [10]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62, customer_id#80, year_total#81] +(78) Project [codegen id : 16] +Output [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40] +Input [10]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40, year_total#41, year_total#60, customer_id#78, year_total#79] -(71) TakeOrderedAndProject -Input [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] -Arguments: 100, [customer_id#38 ASC NULLS FIRST, customer_first_name#39 ASC NULLS FIRST, customer_last_name#40 ASC NULLS FIRST, customer_email_address#41 ASC NULLS FIRST], [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] +(79) TakeOrderedAndProject +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40] +Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_first_name#38 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST, customer_email_address#40 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (83) ++- * ColumnarToRow (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) (unknown) Scan parquet spark_catalog.default.date_dim @@ -434,44 +458,44 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(81) CometFilter Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(74) ColumnarToRow [codegen id : 1] +(82) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(75) BroadcastExchange +(83) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 +BroadcastExchange (87) ++- * ColumnarToRow (86) + +- CometFilter (85) + +- CometScan parquet spark_catalog.default.date_dim (84) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#34, d_year#35] +Output [2]: [d_date_sk#33, d_year#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter -Input [2]: [d_date_sk#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) +(85) CometFilter +Input [2]: [d_date_sk#33, d_year#34] +Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) -(78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_year#35] +(86) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#33, d_year#34] -(79) BroadcastExchange -Input [2]: [d_date_sk#34, d_year#35] +(87) BroadcastExchange +Input [2]: [d_date_sk#33, d_year#34] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#33 +Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#32 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index 0a30aba051..6e42d76f1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -5,43 +5,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [year_total] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + RowToColumnar + WholeStageCodegen (3) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -52,71 +22,109 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + ColumnarToRow + InputAdapter + CometFilter [year_total] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + ColumnarToRow InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index f3c5c46098..c60f2045af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.web_sales @@ -64,7 +66,7 @@ Join condition: None Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] +(10) ReusedExchange [Reuses operator id: 27] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -84,66 +86,70 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(14) RowToColumnar Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(15) CometColumnarExchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] -Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16] -(16) Exchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(17) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(18) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(17) Sort [codegen id : 5] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [i_class#9 ASC NULLS FIRST], false, 0 +(19) ColumnarToRow [codegen id : 4] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] -(18) Window -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(20) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(19) Project [codegen id : 6] -Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] -Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] +(21) Project [codegen id : 5] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] -(20) TakeOrderedAndProject -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +(22) TakeOrderedAndProject +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +Output [2]: [d_date_sk#11, d_date#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#18] +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#18] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 1bc2538b48..03091f0fbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -1,16 +1,16 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (5) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - Sort [i_class] + WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometSort [i_class] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 6fdb365c5b..377f8ee690 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -1,88 +1,94 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * BroadcastHashJoin Inner BuildRight (83) - :- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * ColumnarToRow (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) - : : +- BroadcastExchange (57) - : : +- * BroadcastHashJoin LeftSemi BuildRight (56) - : : :- * ColumnarToRow (54) - : : : +- CometFilter (53) - : : : +- CometScan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - +- BroadcastExchange (82) - +- * Filter (81) - +- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * BroadcastHashJoin Inner BuildRight (76) - :- * Project (74) - : +- * BroadcastHashJoin Inner BuildRight (73) - : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : :- * ColumnarToRow (69) - : : : +- CometFilter (68) - : : : +- CometScan parquet spark_catalog.default.store_sales (67) - : : +- ReusedExchange (70) - : +- ReusedExchange (72) - +- ReusedExchange (75) +TakeOrderedAndProject (90) ++- * BroadcastHashJoin Inner BuildRight (89) + :- * ColumnarToRow (70) + : +- CometFilter (69) + : +- CometHashAggregate (68) + : +- CometColumnarExchange (67) + : +- RowToColumnar (66) + : +- * HashAggregate (65) + : +- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * BroadcastHashJoin LeftSemi BuildRight (53) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (52) + : : : +- * Project (51) + : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : :- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (49) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) + : : : :- * ColumnarToRow (37) + : : : : +- CometHashAggregate (36) + : : : : +- CometColumnarExchange (35) + : : : : +- RowToColumnar (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * ColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (47) + : : : +- * Project (46) + : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : :- * Project (43) + : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : :- * ColumnarToRow (40) + : : : : : +- CometFilter (39) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) + : : : : +- ReusedExchange (41) + : : : +- ReusedExchange (44) + : : +- BroadcastExchange (59) + : : +- * BroadcastHashJoin LeftSemi BuildRight (58) + : : :- * ColumnarToRow (56) + : : : +- CometFilter (55) + : : : +- CometScan parquet spark_catalog.default.item (54) + : : +- ReusedExchange (57) + : +- ReusedExchange (62) + +- BroadcastExchange (88) + +- * ColumnarToRow (87) + +- CometFilter (86) + +- CometHashAggregate (85) + +- CometColumnarExchange (84) + +- RowToColumnar (83) + +- * HashAggregate (82) + +- * Project (81) + +- * BroadcastHashJoin Inner BuildRight (80) + :- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * BroadcastHashJoin LeftSemi BuildRight (75) + : : :- * ColumnarToRow (73) + : : : +- CometFilter (72) + : : : +- CometScan parquet spark_catalog.default.store_sales (71) + : : +- ReusedExchange (74) + : +- ReusedExchange (76) + +- ReusedExchange (79) (unknown) Scan parquet spark_catalog.default.store_sales @@ -186,7 +192,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 117] +(22) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -223,7 +229,7 @@ Join condition: None Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 117] +(30) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 6] @@ -243,16 +249,20 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) Exchange +(34) RowToColumnar Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) HashAggregate [codegen id : 10] +(35) CometColumnarExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(36) CometHashAggregate Input [3]: [brand_id#26, class_id#27, category_id#28] Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(37) ColumnarToRow [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] (unknown) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] @@ -262,68 +272,68 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(39) CometFilter Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] Condition : isnotnull(ws_item_sk#29) -(38) ColumnarToRow [codegen id : 9] +(40) ColumnarToRow [codegen id : 9] Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -(39) ReusedExchange [Reuses operator id: 19] +(41) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(40) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_item_sk#29] Right keys [1]: [i_item_sk#32] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(43) Project [codegen id : 9] Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(42) ReusedExchange [Reuses operator id: 117] +(44) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#36] -(43) BroadcastHashJoin [codegen id : 9] +(45) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#30] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(44) Project [codegen id : 9] +(46) Project [codegen id : 9] Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] -(45) BroadcastExchange +(47) BroadcastExchange Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(46) BroadcastHashJoin [codegen id : 10] +(48) BroadcastHashJoin [codegen id : 10] Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] Join type: LeftSemi Join condition: None -(47) BroadcastExchange +(49) BroadcastExchange Input [3]: [brand_id#26, class_id#27, category_id#28] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(48) BroadcastHashJoin [codegen id : 11] +(50) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#26, class_id#27, category_id#28] Join type: Inner Join condition: None -(49) Project [codegen id : 11] +(51) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#37] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [ss_item_sk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 25] +(53) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi @@ -336,420 +346,434 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) -(54) ColumnarToRow [codegen id : 23] +(56) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(55) ReusedExchange [Reuses operator id: 50] +(57) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(56) BroadcastHashJoin [codegen id : 23] +(58) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#38] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(57) BroadcastExchange +(59) BroadcastExchange Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(58) BroadcastHashJoin [codegen id : 25] +(60) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(59) Project [codegen id : 25] +(61) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(60) ReusedExchange [Reuses operator id: 108] +(62) ReusedExchange [Reuses operator id: 116] Output [1]: [d_date_sk#42] -(61) BroadcastHashJoin [codegen id : 25] +(63) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#42] Join type: Inner Join condition: None -(62) Project [codegen id : 25] +(64) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(63) HashAggregate [codegen id : 25] +(65) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(64) Exchange +(66) RowToColumnar +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(67) CometColumnarExchange Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(65) HashAggregate [codegen id : 52] +(68) CometHashAggregate Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(66) Filter [codegen id : 52] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(69) CometFilter +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] +Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(70) ColumnarToRow [codegen id : 52] +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_sold_date_sk#57 IN dynamicpruning#58)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(68) CometFilter -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] -Condition : isnotnull(ss_item_sk#56) +(72) CometFilter +Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] +Condition : isnotnull(ss_item_sk#54) -(69) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +(73) ColumnarToRow [codegen id : 50] +Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(70) ReusedExchange [Reuses operator id: 50] +(74) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(71) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] +(75) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(72) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +(76) ReusedExchange [Reuses operator id: 59] +Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [i_item_sk#61] +(77) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#59] Join type: Inner Join condition: None -(74) Project [codegen id : 50] -Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +(78) Project [codegen id : 50] +Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] +Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(75) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#65] +(79) ReusedExchange [Reuses operator id: 130] +Output [1]: [d_date_sk#63] -(76) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#65] +(80) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_sold_date_sk#57] +Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(77) Project [codegen id : 50] -Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] - -(78) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] -Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] - -(79) Exchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(80) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72, count(1)#73] -Results [6]: [store AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] - -(81) Filter [codegen id : 51] -Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) - -(82) BroadcastExchange -Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] +(81) Project [codegen id : 50] +Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] +Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] + +(82) HashAggregate [codegen id : 50] +Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] +Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] +Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] + +(83) RowToColumnar +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] + +(84) CometColumnarExchange +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(85) CometHashAggregate +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] + +(86) CometFilter +Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +Condition : (isnotnull(sales#71) AND (cast(sales#71 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(87) ColumnarToRow [codegen id : 51] +Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] + +(88) BroadcastExchange +Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(83) BroadcastHashJoin [codegen id : 52] +(89) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Right keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Join type: Inner Join condition: None -(84) TakeOrderedAndProject -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] +(90) TakeOrderedAndProject +Input [12]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51, channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51, channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (103) -+- Exchange (102) - +- * HashAggregate (101) - +- Union (100) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * ColumnarToRow (86) - : : +- CometScan parquet spark_catalog.default.store_sales (85) - : +- ReusedExchange (87) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * ColumnarToRow (91) - : : +- CometScan parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (92) - +- * Project (99) - +- * BroadcastHashJoin Inner BuildRight (98) - :- * ColumnarToRow (96) - : +- CometScan parquet spark_catalog.default.web_sales (95) - +- ReusedExchange (97) +Subquery:1 Hosting operator id = 69 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* ColumnarToRow (111) ++- CometHashAggregate (110) + +- CometColumnarExchange (109) + +- RowToColumnar (108) + +- * HashAggregate (107) + +- Union (106) + :- * Project (95) + : +- * BroadcastHashJoin Inner BuildRight (94) + : :- * ColumnarToRow (92) + : : +- CometScan parquet spark_catalog.default.store_sales (91) + : +- ReusedExchange (93) + :- * Project (100) + : +- * BroadcastHashJoin Inner BuildRight (99) + : :- * ColumnarToRow (97) + : : +- CometScan parquet spark_catalog.default.catalog_sales (96) + : +- ReusedExchange (98) + +- * Project (105) + +- * BroadcastHashJoin Inner BuildRight (104) + :- * ColumnarToRow (102) + : +- CometScan parquet spark_catalog.default.web_sales (101) + +- ReusedExchange (103) (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] +Output [3]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#79), dynamicpruningexpression(ss_sold_date_sk#79 IN dynamicpruning#80)] +PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_sold_date_sk#75 IN dynamicpruning#76)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] +(92) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75] -(87) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#81] +(93) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#77] -(88) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#79] -Right keys [1]: [d_date_sk#81] +(94) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#75] +Right keys [1]: [d_date_sk#77] Join type: Inner Join condition: None -(89) Project [codegen id : 2] -Output [2]: [ss_quantity#77 AS quantity#82, ss_list_price#78 AS list_price#83] -Input [4]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79, d_date_sk#81] +(95) Project [codegen id : 2] +Output [2]: [ss_quantity#73 AS quantity#78, ss_list_price#74 AS list_price#79] +Input [4]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75, d_date_sk#77] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] +Output [3]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#86), dynamicpruningexpression(cs_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(cs_sold_date_sk#82), dynamicpruningexpression(cs_sold_date_sk#82 IN dynamicpruning#83)] ReadSchema: struct -(91) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] +(97) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82] -(92) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#88] +(98) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#84] -(93) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#86] -Right keys [1]: [d_date_sk#88] +(99) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#82] +Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(94) Project [codegen id : 4] -Output [2]: [cs_quantity#84 AS quantity#89, cs_list_price#85 AS list_price#90] -Input [4]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86, d_date_sk#88] +(100) Project [codegen id : 4] +Output [2]: [cs_quantity#80 AS quantity#85, cs_list_price#81 AS list_price#86] +Input [4]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82, d_date_sk#84] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] +PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#90)] ReadSchema: struct -(96) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +(102) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] -(97) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#95] +(103) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#91] -(98) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#93] -Right keys [1]: [d_date_sk#95] +(104) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#89] +Right keys [1]: [d_date_sk#91] Join type: Inner Join condition: None -(99) Project [codegen id : 6] -Output [2]: [ws_quantity#91 AS quantity#96, ws_list_price#92 AS list_price#97] -Input [4]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93, d_date_sk#95] +(105) Project [codegen id : 6] +Output [2]: [ws_quantity#87 AS quantity#92, ws_list_price#88 AS list_price#93] +Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#91] -(100) Union +(106) Union -(101) HashAggregate [codegen id : 7] -Input [2]: [quantity#82, list_price#83] +(107) HashAggregate [codegen id : 7] +Input [2]: [quantity#78, list_price#79] Keys: [] -Functions [1]: [partial_avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] -Aggregate Attributes [2]: [sum#98, count#99] -Results [2]: [sum#100, count#101] +Functions [1]: [partial_avg((cast(quantity#78 as decimal(10,0)) * list_price#79))] +Aggregate Attributes [2]: [sum#94, count#95] +Results [2]: [sum#96, count#97] -(102) Exchange -Input [2]: [sum#100, count#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(108) RowToColumnar +Input [2]: [sum#96, count#97] -(103) HashAggregate [codegen id : 8] -Input [2]: [sum#100, count#101] +(109) CometColumnarExchange +Input [2]: [sum#96, count#97] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(110) CometHashAggregate +Input [2]: [sum#96, count#97] Keys: [] -Functions [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] -Aggregate Attributes [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102] -Results [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102 AS average_sales#103] +Functions [1]: [avg((cast(quantity#78 as decimal(10,0)) * list_price#79))] + +(111) ColumnarToRow [codegen id : 8] +Input [1]: [average_sales#98] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#79 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#75 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#82 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * ColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan parquet spark_catalog.default.date_dim (104) +BroadcastExchange (116) ++- * ColumnarToRow (115) + +- CometProject (114) + +- CometFilter (113) + +- CometScan parquet spark_catalog.default.date_dim (112) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#42, d_week_seq#104] +Output [2]: [d_date_sk#42, d_week_seq#99] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(105) CometFilter -Input [2]: [d_date_sk#42, d_week_seq#104] -Condition : ((isnotnull(d_week_seq#104) AND (d_week_seq#104 = Subquery scalar-subquery#105, [id=#106])) AND isnotnull(d_date_sk#42)) +(113) CometFilter +Input [2]: [d_date_sk#42, d_week_seq#99] +Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#42)) -(106) CometProject -Input [2]: [d_date_sk#42, d_week_seq#104] +(114) CometProject +Input [2]: [d_date_sk#42, d_week_seq#99] Arguments: [d_date_sk#42], [d_date_sk#42] -(107) ColumnarToRow [codegen id : 1] +(115) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#42] -(108) BroadcastExchange +(116) BroadcastExchange Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 105 Hosting Expression = Subquery scalar-subquery#105, [id=#106] -* ColumnarToRow (112) -+- CometProject (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +Subquery:6 Hosting operator id = 113 Hosting Expression = Subquery scalar-subquery#100, [id=#101] +* ColumnarToRow (120) ++- CometProject (119) + +- CometFilter (118) + +- CometScan parquet spark_catalog.default.date_dim (117) (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] +Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(110) CometFilter -Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] -Condition : (((((isnotnull(d_year#108) AND isnotnull(d_moy#109)) AND isnotnull(d_dom#110)) AND (d_year#108 = 1999)) AND (d_moy#109 = 12)) AND (d_dom#110 = 16)) +(118) CometFilter +Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] +Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 1999)) AND (d_moy#104 = 12)) AND (d_dom#105 = 16)) -(111) CometProject -Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] -Arguments: [d_week_seq#107], [d_week_seq#107] +(119) CometProject +Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] +Arguments: [d_week_seq#102], [d_week_seq#102] -(112) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#107] +(120) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#102] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (125) ++- * ColumnarToRow (124) + +- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#111] +Output [2]: [d_date_sk#25, d_year#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter -Input [2]: [d_date_sk#25, d_year#111] -Condition : (((isnotnull(d_year#111) AND (d_year#111 >= 1998)) AND (d_year#111 <= 2000)) AND isnotnull(d_date_sk#25)) +(122) CometFilter +Input [2]: [d_date_sk#25, d_year#106] +Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1998)) AND (d_year#106 <= 2000)) AND isnotnull(d_date_sk#25)) -(115) CometProject -Input [2]: [d_date_sk#25, d_year#111] +(123) CometProject +Input [2]: [d_date_sk#25, d_year#106] Arguments: [d_date_sk#25], [d_date_sk#25] -(116) ColumnarToRow [codegen id : 1] +(124) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(117) BroadcastExchange +(125) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:10 Hosting operator id = 86 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 -BroadcastExchange (122) -+- * ColumnarToRow (121) - +- CometProject (120) - +- CometFilter (119) - +- CometScan parquet spark_catalog.default.date_dim (118) +Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 +BroadcastExchange (130) ++- * ColumnarToRow (129) + +- CometProject (128) + +- CometFilter (127) + +- CometScan parquet spark_catalog.default.date_dim (126) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#112] +Output [2]: [d_date_sk#63, d_week_seq#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(119) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#112] -Condition : ((isnotnull(d_week_seq#112) AND (d_week_seq#112 = Subquery scalar-subquery#113, [id=#114])) AND isnotnull(d_date_sk#65)) +(127) CometFilter +Input [2]: [d_date_sk#63, d_week_seq#107] +Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#63)) -(120) CometProject -Input [2]: [d_date_sk#65, d_week_seq#112] -Arguments: [d_date_sk#65], [d_date_sk#65] +(128) CometProject +Input [2]: [d_date_sk#63, d_week_seq#107] +Arguments: [d_date_sk#63], [d_date_sk#63] -(121) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] +(129) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#63] -(122) BroadcastExchange -Input [1]: [d_date_sk#65] +(130) BroadcastExchange +Input [1]: [d_date_sk#63] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:12 Hosting operator id = 119 Hosting Expression = Subquery scalar-subquery#113, [id=#114] -* ColumnarToRow (126) -+- CometProject (125) - +- CometFilter (124) - +- CometScan parquet spark_catalog.default.date_dim (123) +Subquery:12 Hosting operator id = 127 Hosting Expression = Subquery scalar-subquery#108, [id=#109] +* ColumnarToRow (134) ++- CometProject (133) + +- CometFilter (132) + +- CometScan parquet spark_catalog.default.date_dim (131) (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] +Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(124) CometFilter -Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] -Condition : (((((isnotnull(d_year#116) AND isnotnull(d_moy#117)) AND isnotnull(d_dom#118)) AND (d_year#116 = 1998)) AND (d_moy#117 = 12)) AND (d_dom#118 = 16)) +(132) CometFilter +Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] +Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1998)) AND (d_moy#112 = 12)) AND (d_dom#113 = 16)) -(125) CometProject -Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] -Arguments: [d_week_seq#115], [d_week_seq#115] +(133) CometProject +Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] +Arguments: [d_week_seq#110], [d_week_seq#110] -(126) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#115] +(134) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#110] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 09d8d9dde3..13ac319fbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -1,202 +1,210 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (52) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #4 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - InputAdapter - Exchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [sales] + Subquery #4 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometHashAggregate [sum,count] + CometColumnarExchange #12 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + RowToColumnar + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - ColumnarToRow - InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #5 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #6 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 InputAdapter ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #3 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 WholeStageCodegen (51) - Filter [sales] - ReusedSubquery [average_sales] #4 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow + ColumnarToRow + InputAdapter + CometFilter [sales] + ReusedSubquery [average_sales] #4 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 + RowToColumnar + WholeStageCodegen (50) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + BroadcastExchange #15 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + Subquery #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + ReusedExchange [ss_item_sk] #3 InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] - Subquery #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #15 + ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index a8db177f87..0085b3b4c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -1,129 +1,135 @@ == Physical Plan == -TakeOrderedAndProject (125) -+- * HashAggregate (124) - +- Exchange (123) - +- * HashAggregate (122) - +- Union (121) - :- * HashAggregate (100) - : +- Exchange (99) - : +- * HashAggregate (98) - : +- Union (97) - : :- * Filter (66) - : : +- * HashAggregate (65) - : : +- Exchange (64) - : : +- * HashAggregate (63) - : : +- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (51) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (50) - : : : : +- * Project (49) - : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (47) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : : :- * HashAggregate (35) - : : : : : +- Exchange (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * ColumnarToRow (9) - : : : : : : : +- CometFilter (8) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * ColumnarToRow (12) - : : : : : : : +- CometFilter (11) - : : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * ColumnarToRow (15) - : : : : : : : : +- CometFilter (14) - : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (45) - : : : : +- * Project (44) - : : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : : :- * Project (41) - : : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : : :- * ColumnarToRow (38) - : : : : : : +- CometFilter (37) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : : +- ReusedExchange (39) - : : : : +- ReusedExchange (42) - : : : +- BroadcastExchange (57) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (56) - : : : :- * ColumnarToRow (54) - : : : : +- CometFilter (53) - : : : : +- CometScan parquet spark_catalog.default.item (52) - : : : +- ReusedExchange (55) - : : +- ReusedExchange (60) - : :- * Filter (81) - : : +- * HashAggregate (80) - : : +- Exchange (79) - : : +- * HashAggregate (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : : : :- * ColumnarToRow (69) - : : : : : +- CometFilter (68) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (67) - : : : : +- ReusedExchange (70) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- * Filter (96) - : +- * HashAggregate (95) - : +- Exchange (94) - : +- * HashAggregate (93) - : +- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * Project (89) - : : +- * BroadcastHashJoin Inner BuildRight (88) - : : :- * BroadcastHashJoin LeftSemi BuildRight (86) - : : : :- * ColumnarToRow (84) - : : : : +- CometFilter (83) - : : : : +- CometScan parquet spark_catalog.default.web_sales (82) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (87) - : +- ReusedExchange (90) - :- * HashAggregate (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- * HashAggregate (102) - : +- ReusedExchange (101) - :- * HashAggregate (110) - : +- Exchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- ReusedExchange (106) - :- * HashAggregate (115) - : +- Exchange (114) - : +- * HashAggregate (113) - : +- * HashAggregate (112) - : +- ReusedExchange (111) - +- * HashAggregate (120) - +- Exchange (119) - +- * HashAggregate (118) - +- * HashAggregate (117) - +- ReusedExchange (116) +* ColumnarToRow (131) ++- CometTakeOrderedAndProject (130) + +- CometHashAggregate (129) + +- CometColumnarExchange (128) + +- CometHashAggregate (127) + +- CometUnion (126) + :- CometHashAggregate (105) + : +- CometColumnarExchange (104) + : +- CometHashAggregate (103) + : +- CometUnion (102) + : :- CometFilter (69) + : : +- CometHashAggregate (68) + : : +- CometColumnarExchange (67) + : : +- RowToColumnar (66) + : : +- * HashAggregate (65) + : : +- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (61) + : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (53) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (52) + : : : : +- * Project (51) + : : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : : +- BroadcastExchange (49) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) + : : : : :- * ColumnarToRow (37) + : : : : : +- CometHashAggregate (36) + : : : : : +- CometColumnarExchange (35) + : : : : : +- RowToColumnar (34) + : : : : : +- * HashAggregate (33) + : : : : : +- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * ColumnarToRow (9) + : : : : : : : +- CometFilter (8) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : : +- BroadcastExchange (27) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : : :- * ColumnarToRow (12) + : : : : : : : +- CometFilter (11) + : : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : : +- BroadcastExchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * ColumnarToRow (15) + : : : : : : : : +- CometFilter (14) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * ColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : : +- ReusedExchange (22) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (47) + : : : : +- * Project (46) + : : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : : :- * Project (43) + : : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : : :- * ColumnarToRow (40) + : : : : : : +- CometFilter (39) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) + : : : : : +- ReusedExchange (41) + : : : : +- ReusedExchange (44) + : : : +- BroadcastExchange (59) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (58) + : : : :- * ColumnarToRow (56) + : : : : +- CometFilter (55) + : : : : +- CometScan parquet spark_catalog.default.item (54) + : : : +- ReusedExchange (57) + : : +- ReusedExchange (62) + : :- CometFilter (85) + : : +- CometHashAggregate (84) + : : +- CometColumnarExchange (83) + : : +- RowToColumnar (82) + : : +- * HashAggregate (81) + : : +- * Project (80) + : : +- * BroadcastHashJoin Inner BuildRight (79) + : : :- * Project (77) + : : : +- * BroadcastHashJoin Inner BuildRight (76) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (74) + : : : : :- * ColumnarToRow (72) + : : : : : +- CometFilter (71) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (70) + : : : : +- ReusedExchange (73) + : : : +- ReusedExchange (75) + : : +- ReusedExchange (78) + : +- CometFilter (101) + : +- CometHashAggregate (100) + : +- CometColumnarExchange (99) + : +- RowToColumnar (98) + : +- * HashAggregate (97) + : +- * Project (96) + : +- * BroadcastHashJoin Inner BuildRight (95) + : :- * Project (93) + : : +- * BroadcastHashJoin Inner BuildRight (92) + : : :- * BroadcastHashJoin LeftSemi BuildRight (90) + : : : :- * ColumnarToRow (88) + : : : : +- CometFilter (87) + : : : : +- CometScan parquet spark_catalog.default.web_sales (86) + : : : +- ReusedExchange (89) + : : +- ReusedExchange (91) + : +- ReusedExchange (94) + :- CometHashAggregate (110) + : +- CometColumnarExchange (109) + : +- CometHashAggregate (108) + : +- CometHashAggregate (107) + : +- ReusedExchange (106) + :- CometHashAggregate (115) + : +- CometColumnarExchange (114) + : +- CometHashAggregate (113) + : +- CometHashAggregate (112) + : +- ReusedExchange (111) + :- CometHashAggregate (120) + : +- CometColumnarExchange (119) + : +- CometHashAggregate (118) + : +- CometHashAggregate (117) + : +- ReusedExchange (116) + +- CometHashAggregate (125) + +- CometColumnarExchange (124) + +- CometHashAggregate (123) + +- CometHashAggregate (122) + +- ReusedExchange (121) (unknown) Scan parquet spark_catalog.default.store_sales @@ -227,7 +233,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 159] +(22) ReusedExchange [Reuses operator id: 167] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -264,7 +270,7 @@ Join condition: None Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 159] +(30) ReusedExchange [Reuses operator id: 167] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 6] @@ -284,16 +290,20 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) Exchange +(34) RowToColumnar Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) HashAggregate [codegen id : 10] +(35) CometColumnarExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(36) CometHashAggregate Input [3]: [brand_id#26, class_id#27, category_id#28] Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(37) ColumnarToRow [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] (unknown) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] @@ -303,68 +313,68 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) CometFilter +(39) CometFilter Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] Condition : isnotnull(ws_item_sk#29) -(38) ColumnarToRow [codegen id : 9] +(40) ColumnarToRow [codegen id : 9] Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -(39) ReusedExchange [Reuses operator id: 19] +(41) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(40) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_item_sk#29] Right keys [1]: [i_item_sk#32] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(43) Project [codegen id : 9] Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(42) ReusedExchange [Reuses operator id: 159] +(44) ReusedExchange [Reuses operator id: 167] Output [1]: [d_date_sk#36] -(43) BroadcastHashJoin [codegen id : 9] +(45) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#30] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(44) Project [codegen id : 9] +(46) Project [codegen id : 9] Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] -(45) BroadcastExchange +(47) BroadcastExchange Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(46) BroadcastHashJoin [codegen id : 10] +(48) BroadcastHashJoin [codegen id : 10] Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] Join type: LeftSemi Join condition: None -(47) BroadcastExchange +(49) BroadcastExchange Input [3]: [brand_id#26, class_id#27, category_id#28] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(48) BroadcastHashJoin [codegen id : 11] +(50) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#26, class_id#27, category_id#28] Join type: Inner Join condition: None -(49) Project [codegen id : 11] +(51) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#37] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [ss_item_sk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 25] +(53) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi @@ -377,588 +387,576 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Condition : isnotnull(i_item_sk#38) -(54) ColumnarToRow [codegen id : 23] +(56) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(55) ReusedExchange [Reuses operator id: 50] +(57) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(56) BroadcastHashJoin [codegen id : 23] +(58) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#38] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(57) BroadcastExchange +(59) BroadcastExchange Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(58) BroadcastHashJoin [codegen id : 25] +(60) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(59) Project [codegen id : 25] +(61) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(60) ReusedExchange [Reuses operator id: 154] +(62) ReusedExchange [Reuses operator id: 162] Output [1]: [d_date_sk#42] -(61) BroadcastHashJoin [codegen id : 25] +(63) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#42] Join type: Inner Join condition: None -(62) Project [codegen id : 25] +(64) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(63) HashAggregate [codegen id : 25] +(65) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(64) Exchange +(66) RowToColumnar +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(67) CometColumnarExchange Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(65) HashAggregate [codegen id : 26] +(68) CometHashAggregate Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(66) Filter [codegen id : 26] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(69) CometFilter +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] +Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#59), dynamicpruningexpression(cs_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(68) CometFilter -Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] -Condition : isnotnull(cs_item_sk#56) +(71) CometFilter +Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Condition : isnotnull(cs_item_sk#54) -(69) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +(72) ColumnarToRow [codegen id : 50] +Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -(70) ReusedExchange [Reuses operator id: 50] +(73) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(71) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#56] +(74) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [cs_item_sk#54] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(72) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +(75) ReusedExchange [Reuses operator id: 59] +Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(73) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#56] -Right keys [1]: [i_item_sk#61] +(76) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [cs_item_sk#54] +Right keys [1]: [i_item_sk#59] Join type: Inner Join condition: None -(74) Project [codegen id : 51] -Output [6]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [8]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +(77) Project [codegen id : 50] +Output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] +Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(75) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#65] +(78) ReusedExchange [Reuses operator id: 162] +Output [1]: [d_date_sk#63] -(76) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#59] -Right keys [1]: [d_date_sk#65] +(79) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [cs_sold_date_sk#57] +Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(77) Project [codegen id : 51] -Output [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [7]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] +(80) Project [codegen id : 50] +Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] +Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] + +(81) HashAggregate [codegen id : 50] +Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] +Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] +Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -(78) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] -Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +(82) RowToColumnar +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -(79) Exchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(83) CometColumnarExchange +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(80) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#72, count(1)#73] -Results [6]: [catalog AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] +(84) CometHashAggregate +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] -(81) Filter [codegen id : 52] -Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(85) CometFilter +Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +Condition : (isnotnull(sales#71) AND (cast(sales#71 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] +Output [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#81)] +PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#77)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(83) CometFilter -Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] -Condition : isnotnull(ws_item_sk#77) +(87) CometFilter +Input [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] +Condition : isnotnull(ws_item_sk#73) -(84) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] +(88) ColumnarToRow [codegen id : 75] +Input [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] -(85) ReusedExchange [Reuses operator id: 50] +(89) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(86) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#77] +(90) BroadcastHashJoin [codegen id : 75] +Left keys [1]: [ws_item_sk#73] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(87) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] +(91) ReusedExchange [Reuses operator id: 59] +Output [4]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] -(88) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#77] -Right keys [1]: [i_item_sk#82] +(92) BroadcastHashJoin [codegen id : 75] +Left keys [1]: [ws_item_sk#73] +Right keys [1]: [i_item_sk#78] Join type: Inner Join condition: None -(89) Project [codegen id : 77] -Output [6]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#83, i_class_id#84, i_category_id#85] -Input [8]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] +(93) Project [codegen id : 75] +Output [6]: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81] +Input [8]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] -(90) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#86] +(94) ReusedExchange [Reuses operator id: 162] +Output [1]: [d_date_sk#82] -(91) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#86] +(95) BroadcastHashJoin [codegen id : 75] +Left keys [1]: [ws_sold_date_sk#76] +Right keys [1]: [d_date_sk#82] Join type: Inner Join condition: None -(92) Project [codegen id : 77] -Output [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] -Input [7]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#83, i_class_id#84, i_category_id#85, d_date_sk#86] - -(93) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] -Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] -Functions [2]: [partial_sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), partial_count(1)] -Aggregate Attributes [3]: [sum#87, isEmpty#88, count#89] -Results [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] - -(94) Exchange -Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] -Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, [plan_id=11] - -(95) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] -Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] -Functions [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#93, count(1)#94] -Results [6]: [web AS channel#95, i_brand_id#83, i_class_id#84, i_category_id#85, sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#93 AS sales#96, count(1)#94 AS number_sales#97] - -(96) Filter [codegen id : 78] -Input [6]: [channel#95, i_brand_id#83, i_class_id#84, i_category_id#85, sales#96, number_sales#97] -Condition : (isnotnull(sales#96) AND (cast(sales#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) - -(97) Union - -(98) HashAggregate [codegen id : 79] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] -Aggregate Attributes [3]: [sum#98, isEmpty#99, sum#100] -Results [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(99) Exchange -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(100) HashAggregate [codegen id : 80] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] - -(101) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(102) HashAggregate [codegen id : 160] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] -Results [5]: [channel#51, i_brand_id#39, i_class_id#40, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] - -(103) HashAggregate [codegen id : 160] -Input [5]: [channel#51, i_brand_id#39, i_class_id#40, sum_sales#106, number_sales#107] -Keys [3]: [channel#51, i_brand_id#39, i_class_id#40] -Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] -Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] - -(104) Exchange -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(105) HashAggregate [codegen id : 161] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] -Keys [3]: [channel#51, i_brand_id#39, i_class_id#40] -Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] -Aggregate Attributes [2]: [sum(sum_sales#106)#114, sum(number_sales#107)#115] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, null AS i_category_id#116, sum(sum_sales#106)#114 AS sum(sum_sales)#117, sum(number_sales#107)#115 AS sum(number_sales)#118] - -(106) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(107) HashAggregate [codegen id : 241] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] -Results [4]: [channel#51, i_brand_id#39, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] - -(108) HashAggregate [codegen id : 241] -Input [4]: [channel#51, i_brand_id#39, sum_sales#106, number_sales#107] -Keys [2]: [channel#51, i_brand_id#39] -Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] -Aggregate Attributes [3]: [sum#119, isEmpty#120, sum#121] -Results [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] - -(109) Exchange -Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] -Arguments: hashpartitioning(channel#51, i_brand_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=14] - -(110) HashAggregate [codegen id : 242] -Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] -Keys [2]: [channel#51, i_brand_id#39] -Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] -Aggregate Attributes [2]: [sum(sum_sales#106)#125, sum(number_sales#107)#126] -Results [6]: [channel#51, i_brand_id#39, null AS i_class_id#127, null AS i_category_id#128, sum(sum_sales#106)#125 AS sum(sum_sales)#129, sum(number_sales#107)#126 AS sum(number_sales)#130] - -(111) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(112) HashAggregate [codegen id : 322] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] -Results [3]: [channel#51, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] - -(113) HashAggregate [codegen id : 322] -Input [3]: [channel#51, sum_sales#106, number_sales#107] -Keys [1]: [channel#51] -Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] -Aggregate Attributes [3]: [sum#131, isEmpty#132, sum#133] -Results [4]: [channel#51, sum#134, isEmpty#135, sum#136] - -(114) Exchange -Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] -Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(115) HashAggregate [codegen id : 323] -Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] -Keys [1]: [channel#51] -Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] -Aggregate Attributes [2]: [sum(sum_sales#106)#137, sum(number_sales#107)#138] -Results [6]: [channel#51, null AS i_brand_id#139, null AS i_class_id#140, null AS i_category_id#141, sum(sum_sales#106)#137 AS sum(sum_sales)#142, sum(number_sales#107)#138 AS sum(number_sales)#143] - -(116) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(117) HashAggregate [codegen id : 403] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] -Results [2]: [sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] - -(118) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#106, number_sales#107] +(96) Project [codegen id : 75] +Output [5]: [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81] +Input [7]: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, d_date_sk#82] + +(97) HashAggregate [codegen id : 75] +Input [5]: [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81] +Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] +Functions [2]: [partial_sum((cast(ws_quantity#74 as decimal(10,0)) * ws_list_price#75)), partial_count(1)] +Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] +Results [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] + +(98) RowToColumnar +Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] + +(99) CometColumnarExchange +Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] +Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(100) CometHashAggregate +Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] +Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] +Functions [2]: [sum((cast(ws_quantity#74 as decimal(10,0)) * ws_list_price#75)), count(1)] + +(101) CometFilter +Input [6]: [channel#89, i_brand_id#79, i_class_id#80, i_category_id#81, sales#90, number_sales#91] +Condition : (isnotnull(sales#90) AND (cast(sales#90 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(102) CometUnion +Child 0 Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] +Child 1 Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +Child 2 Input [6]: [channel#89, i_brand_id#79, i_class_id#80, i_category_id#81, sales#90, number_sales#91] + +(103) CometHashAggregate +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] +Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] + +(104) CometColumnarExchange +Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] +Arguments: hashpartitioning(channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(105) CometHashAggregate +Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] +Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(106) ReusedExchange [Reuses operator id: 104] +Output [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] + +(107) CometHashAggregate +Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] +Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(108) CometHashAggregate +Input [5]: [channel#49, i_brand_id#39, i_class_id#40, sum_sales#95, number_sales#96] +Keys [3]: [channel#49, i_brand_id#39, i_class_id#40] +Functions [2]: [partial_sum(sum_sales#95), partial_sum(number_sales#96)] + +(109) CometColumnarExchange +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, sum#97, isEmpty#98, sum#99] +Arguments: hashpartitioning(channel#49, i_brand_id#39, i_class_id#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(110) CometHashAggregate +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, sum#97, isEmpty#98, sum#99] +Keys [3]: [channel#49, i_brand_id#39, i_class_id#40] +Functions [2]: [sum(sum_sales#95), sum(number_sales#96)] + +(111) ReusedExchange [Reuses operator id: 104] +Output [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] + +(112) CometHashAggregate +Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] +Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(113) CometHashAggregate +Input [4]: [channel#49, i_brand_id#39, sum_sales#95, number_sales#96] +Keys [2]: [channel#49, i_brand_id#39] +Functions [2]: [partial_sum(sum_sales#95), partial_sum(number_sales#96)] + +(114) CometColumnarExchange +Input [5]: [channel#49, i_brand_id#39, sum#100, isEmpty#101, sum#102] +Arguments: hashpartitioning(channel#49, i_brand_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(115) CometHashAggregate +Input [5]: [channel#49, i_brand_id#39, sum#100, isEmpty#101, sum#102] +Keys [2]: [channel#49, i_brand_id#39] +Functions [2]: [sum(sum_sales#95), sum(number_sales#96)] + +(116) ReusedExchange [Reuses operator id: 104] +Output [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] + +(117) CometHashAggregate +Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] +Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(118) CometHashAggregate +Input [3]: [channel#49, sum_sales#95, number_sales#96] +Keys [1]: [channel#49] +Functions [2]: [partial_sum(sum_sales#95), partial_sum(number_sales#96)] + +(119) CometColumnarExchange +Input [4]: [channel#49, sum#103, isEmpty#104, sum#105] +Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] + +(120) CometHashAggregate +Input [4]: [channel#49, sum#103, isEmpty#104, sum#105] +Keys [1]: [channel#49] +Functions [2]: [sum(sum_sales#95), sum(number_sales#96)] + +(121) ReusedExchange [Reuses operator id: 104] +Output [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] + +(122) CometHashAggregate +Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] +Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(123) CometHashAggregate +Input [2]: [sum_sales#95, number_sales#96] Keys: [] -Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] -Aggregate Attributes [3]: [sum#144, isEmpty#145, sum#146] -Results [3]: [sum#147, isEmpty#148, sum#149] +Functions [2]: [partial_sum(sum_sales#95), partial_sum(number_sales#96)] -(119) Exchange -Input [3]: [sum#147, isEmpty#148, sum#149] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] +(124) CometColumnarExchange +Input [3]: [sum#106, isEmpty#107, sum#108] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(120) HashAggregate [codegen id : 404] -Input [3]: [sum#147, isEmpty#148, sum#149] +(125) CometHashAggregate +Input [3]: [sum#106, isEmpty#107, sum#108] Keys: [] -Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] -Aggregate Attributes [2]: [sum(sum_sales#106)#150, sum(number_sales#107)#151] -Results [6]: [null AS channel#152, null AS i_brand_id#153, null AS i_class_id#154, null AS i_category_id#155, sum(sum_sales#106)#150 AS sum(sum_sales)#156, sum(number_sales#107)#151 AS sum(number_sales)#157] - -(121) Union - -(122) HashAggregate [codegen id : 405] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +Functions [2]: [sum(sum_sales#95), sum(number_sales#96)] + +(126) CometUnion +Child 0 Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] +Child 1 Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#109, sum(sum_sales)#110, sum(number_sales)#111] +Child 2 Input [6]: [channel#49, i_brand_id#39, i_class_id#112, i_category_id#113, sum(sum_sales)#114, sum(number_sales)#115] +Child 3 Input [6]: [channel#49, i_brand_id#116, i_class_id#117, i_category_id#118, sum(sum_sales)#119, sum(number_sales)#120] +Child 4 Input [6]: [channel#121, i_brand_id#122, i_class_id#123, i_category_id#124, sum(sum_sales)#125, sum(number_sales)#126] + +(127) CometHashAggregate +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] +Keys [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] Functions: [] -Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -(123) Exchange -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107, 5), ENSURE_REQUIREMENTS, [plan_id=17] +(128) CometColumnarExchange +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] +Arguments: hashpartitioning(channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] -(124) HashAggregate [codegen id : 406] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +(129) CometHashAggregate +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] +Keys [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] Functions: [] -Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -(125) TakeOrderedAndProject -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -Arguments: 100, [channel#51 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +(130) CometTakeOrderedAndProject +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#49,i_brand_id#39,i_class_id#40,i_category_id#41,sum_sales#95,number_sales#96]), 100, [channel#49 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] + +(131) ColumnarToRow [codegen id : 376] +Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (144) -+- Exchange (143) - +- * HashAggregate (142) - +- Union (141) - :- * Project (130) - : +- * BroadcastHashJoin Inner BuildRight (129) - : :- * ColumnarToRow (127) - : : +- CometScan parquet spark_catalog.default.store_sales (126) - : +- ReusedExchange (128) - :- * Project (135) - : +- * BroadcastHashJoin Inner BuildRight (134) - : :- * ColumnarToRow (132) - : : +- CometScan parquet spark_catalog.default.catalog_sales (131) - : +- ReusedExchange (133) - +- * Project (140) - +- * BroadcastHashJoin Inner BuildRight (139) - :- * ColumnarToRow (137) - : +- CometScan parquet spark_catalog.default.web_sales (136) - +- ReusedExchange (138) +Subquery:1 Hosting operator id = 69 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* ColumnarToRow (152) ++- CometHashAggregate (151) + +- CometColumnarExchange (150) + +- RowToColumnar (149) + +- * HashAggregate (148) + +- Union (147) + :- * Project (136) + : +- * BroadcastHashJoin Inner BuildRight (135) + : :- * ColumnarToRow (133) + : : +- CometScan parquet spark_catalog.default.store_sales (132) + : +- ReusedExchange (134) + :- * Project (141) + : +- * BroadcastHashJoin Inner BuildRight (140) + : :- * ColumnarToRow (138) + : : +- CometScan parquet spark_catalog.default.catalog_sales (137) + : +- ReusedExchange (139) + +- * Project (146) + +- * BroadcastHashJoin Inner BuildRight (145) + :- * ColumnarToRow (143) + : +- CometScan parquet spark_catalog.default.web_sales (142) + +- ReusedExchange (144) (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#158, ss_list_price#159, ss_sold_date_sk#160] +Output [3]: [ss_quantity#127, ss_list_price#128, ss_sold_date_sk#129] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#160), dynamicpruningexpression(ss_sold_date_sk#160 IN dynamicpruning#161)] +PartitionFilters: [isnotnull(ss_sold_date_sk#129), dynamicpruningexpression(ss_sold_date_sk#129 IN dynamicpruning#130)] ReadSchema: struct -(127) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#158, ss_list_price#159, ss_sold_date_sk#160] +(133) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#127, ss_list_price#128, ss_sold_date_sk#129] -(128) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#162] +(134) ReusedExchange [Reuses operator id: 167] +Output [1]: [d_date_sk#131] -(129) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#160] -Right keys [1]: [d_date_sk#162] +(135) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#129] +Right keys [1]: [d_date_sk#131] Join type: Inner Join condition: None -(130) Project [codegen id : 2] -Output [2]: [ss_quantity#158 AS quantity#163, ss_list_price#159 AS list_price#164] -Input [4]: [ss_quantity#158, ss_list_price#159, ss_sold_date_sk#160, d_date_sk#162] +(136) Project [codegen id : 2] +Output [2]: [ss_quantity#127 AS quantity#132, ss_list_price#128 AS list_price#133] +Input [4]: [ss_quantity#127, ss_list_price#128, ss_sold_date_sk#129, d_date_sk#131] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#165, cs_list_price#166, cs_sold_date_sk#167] +Output [3]: [cs_quantity#134, cs_list_price#135, cs_sold_date_sk#136] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#167), dynamicpruningexpression(cs_sold_date_sk#167 IN dynamicpruning#168)] +PartitionFilters: [isnotnull(cs_sold_date_sk#136), dynamicpruningexpression(cs_sold_date_sk#136 IN dynamicpruning#137)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#165, cs_list_price#166, cs_sold_date_sk#167] +(138) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#134, cs_list_price#135, cs_sold_date_sk#136] -(133) ReusedExchange [Reuses operator id: 149] -Output [1]: [d_date_sk#169] +(139) ReusedExchange [Reuses operator id: 157] +Output [1]: [d_date_sk#138] -(134) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#167] -Right keys [1]: [d_date_sk#169] +(140) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#136] +Right keys [1]: [d_date_sk#138] Join type: Inner Join condition: None -(135) Project [codegen id : 4] -Output [2]: [cs_quantity#165 AS quantity#170, cs_list_price#166 AS list_price#171] -Input [4]: [cs_quantity#165, cs_list_price#166, cs_sold_date_sk#167, d_date_sk#169] +(141) Project [codegen id : 4] +Output [2]: [cs_quantity#134 AS quantity#139, cs_list_price#135 AS list_price#140] +Input [4]: [cs_quantity#134, cs_list_price#135, cs_sold_date_sk#136, d_date_sk#138] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#172, ws_list_price#173, ws_sold_date_sk#174] +Output [3]: [ws_quantity#141, ws_list_price#142, ws_sold_date_sk#143] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#174), dynamicpruningexpression(ws_sold_date_sk#174 IN dynamicpruning#175)] +PartitionFilters: [isnotnull(ws_sold_date_sk#143), dynamicpruningexpression(ws_sold_date_sk#143 IN dynamicpruning#144)] ReadSchema: struct -(137) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#172, ws_list_price#173, ws_sold_date_sk#174] +(143) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#141, ws_list_price#142, ws_sold_date_sk#143] -(138) ReusedExchange [Reuses operator id: 149] -Output [1]: [d_date_sk#176] +(144) ReusedExchange [Reuses operator id: 157] +Output [1]: [d_date_sk#145] -(139) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#174] -Right keys [1]: [d_date_sk#176] +(145) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#143] +Right keys [1]: [d_date_sk#145] Join type: Inner Join condition: None -(140) Project [codegen id : 6] -Output [2]: [ws_quantity#172 AS quantity#177, ws_list_price#173 AS list_price#178] -Input [4]: [ws_quantity#172, ws_list_price#173, ws_sold_date_sk#174, d_date_sk#176] +(146) Project [codegen id : 6] +Output [2]: [ws_quantity#141 AS quantity#146, ws_list_price#142 AS list_price#147] +Input [4]: [ws_quantity#141, ws_list_price#142, ws_sold_date_sk#143, d_date_sk#145] -(141) Union +(147) Union -(142) HashAggregate [codegen id : 7] -Input [2]: [quantity#163, list_price#164] +(148) HashAggregate [codegen id : 7] +Input [2]: [quantity#132, list_price#133] Keys: [] -Functions [1]: [partial_avg((cast(quantity#163 as decimal(10,0)) * list_price#164))] -Aggregate Attributes [2]: [sum#179, count#180] -Results [2]: [sum#181, count#182] +Functions [1]: [partial_avg((cast(quantity#132 as decimal(10,0)) * list_price#133))] +Aggregate Attributes [2]: [sum#148, count#149] +Results [2]: [sum#150, count#151] -(143) Exchange -Input [2]: [sum#181, count#182] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] +(149) RowToColumnar +Input [2]: [sum#150, count#151] -(144) HashAggregate [codegen id : 8] -Input [2]: [sum#181, count#182] +(150) CometColumnarExchange +Input [2]: [sum#150, count#151] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] + +(151) CometHashAggregate +Input [2]: [sum#150, count#151] Keys: [] -Functions [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))] -Aggregate Attributes [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))#183] -Results [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))#183 AS average_sales#184] +Functions [1]: [avg((cast(quantity#132 as decimal(10,0)) * list_price#133))] + +(152) ColumnarToRow [codegen id : 8] +Input [1]: [average_sales#152] -Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#160 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#129 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 131 Hosting Expression = cs_sold_date_sk#167 IN dynamicpruning#168 -BroadcastExchange (149) -+- * ColumnarToRow (148) - +- CometProject (147) - +- CometFilter (146) - +- CometScan parquet spark_catalog.default.date_dim (145) +Subquery:3 Hosting operator id = 137 Hosting Expression = cs_sold_date_sk#136 IN dynamicpruning#137 +BroadcastExchange (157) ++- * ColumnarToRow (156) + +- CometProject (155) + +- CometFilter (154) + +- CometScan parquet spark_catalog.default.date_dim (153) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#169, d_year#185] +Output [2]: [d_date_sk#138, d_year#153] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(146) CometFilter -Input [2]: [d_date_sk#169, d_year#185] -Condition : (((isnotnull(d_year#185) AND (d_year#185 >= 1998)) AND (d_year#185 <= 2000)) AND isnotnull(d_date_sk#169)) +(154) CometFilter +Input [2]: [d_date_sk#138, d_year#153] +Condition : (((isnotnull(d_year#153) AND (d_year#153 >= 1998)) AND (d_year#153 <= 2000)) AND isnotnull(d_date_sk#138)) -(147) CometProject -Input [2]: [d_date_sk#169, d_year#185] -Arguments: [d_date_sk#169], [d_date_sk#169] +(155) CometProject +Input [2]: [d_date_sk#138, d_year#153] +Arguments: [d_date_sk#138], [d_date_sk#138] -(148) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#169] +(156) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#138] -(149) BroadcastExchange -Input [1]: [d_date_sk#169] +(157) BroadcastExchange +Input [1]: [d_date_sk#138] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -Subquery:4 Hosting operator id = 136 Hosting Expression = ws_sold_date_sk#174 IN dynamicpruning#168 +Subquery:4 Hosting operator id = 142 Hosting Expression = ws_sold_date_sk#143 IN dynamicpruning#137 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (154) -+- * ColumnarToRow (153) - +- CometProject (152) - +- CometFilter (151) - +- CometScan parquet spark_catalog.default.date_dim (150) +BroadcastExchange (162) ++- * ColumnarToRow (161) + +- CometProject (160) + +- CometFilter (159) + +- CometScan parquet spark_catalog.default.date_dim (158) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#42, d_year#186, d_moy#187] +Output [3]: [d_date_sk#42, d_year#154, d_moy#155] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(151) CometFilter -Input [3]: [d_date_sk#42, d_year#186, d_moy#187] -Condition : ((((isnotnull(d_year#186) AND isnotnull(d_moy#187)) AND (d_year#186 = 2000)) AND (d_moy#187 = 11)) AND isnotnull(d_date_sk#42)) +(159) CometFilter +Input [3]: [d_date_sk#42, d_year#154, d_moy#155] +Condition : ((((isnotnull(d_year#154) AND isnotnull(d_moy#155)) AND (d_year#154 = 2000)) AND (d_moy#155 = 11)) AND isnotnull(d_date_sk#42)) -(152) CometProject -Input [3]: [d_date_sk#42, d_year#186, d_moy#187] +(160) CometProject +Input [3]: [d_date_sk#42, d_year#154, d_moy#155] Arguments: [d_date_sk#42], [d_date_sk#42] -(153) ColumnarToRow [codegen id : 1] +(161) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#42] -(154) BroadcastExchange +(162) BroadcastExchange Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (159) -+- * ColumnarToRow (158) - +- CometProject (157) - +- CometFilter (156) - +- CometScan parquet spark_catalog.default.date_dim (155) +BroadcastExchange (167) ++- * ColumnarToRow (166) + +- CometProject (165) + +- CometFilter (164) + +- CometScan parquet spark_catalog.default.date_dim (163) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#188] +Output [2]: [d_date_sk#25, d_year#156] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(156) CometFilter -Input [2]: [d_date_sk#25, d_year#188] -Condition : (((isnotnull(d_year#188) AND (d_year#188 >= 1999)) AND (d_year#188 <= 2001)) AND isnotnull(d_date_sk#25)) +(164) CometFilter +Input [2]: [d_date_sk#25, d_year#156] +Condition : (((isnotnull(d_year#156) AND (d_year#156 >= 1999)) AND (d_year#156 <= 2001)) AND isnotnull(d_date_sk#25)) -(157) CometProject -Input [2]: [d_date_sk#25, d_year#188] +(165) CometProject +Input [2]: [d_date_sk#25, d_year#156] Arguments: [d_date_sk#25], [d_date_sk#25] -(158) ColumnarToRow [codegen id : 1] +(166) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(159) BroadcastExchange +(167) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:9 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 70 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 101 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index a203f9620b..becceb8614 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -1,261 +1,241 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (406) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (26) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] +WholeStageCodegen (376) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometUnion + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometUnion + CometFilter [sales] + Subquery #3 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometHashAggregate [sum,count] + CometColumnarExchange #14 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter - Exchange #14 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #15 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #15 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #15 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 + RowToColumnar + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #15 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - ColumnarToRow - InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #7 + RowToColumnar + WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter - ReusedExchange [ss_item_sk] #5 + BroadcastExchange #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + ReusedExchange [ss_item_sk] #5 InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + ReusedExchange [d_date_sk] #4 + CometFilter [sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + RowToColumnar + WholeStageCodegen (50) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) - HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - InputAdapter - Exchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) - HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) - HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - InputAdapter - Exchange [channel,i_brand_id] #19 - WholeStageCodegen (241) - HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) - HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - InputAdapter - Exchange [channel] #20 - WholeStageCodegen (322) - HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) - HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - InputAdapter - Exchange #21 - WholeStageCodegen (403) - HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + ReusedExchange [d_date_sk] #4 + CometFilter [sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 + RowToColumnar + WholeStageCodegen (75) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + CometHashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] + CometColumnarExchange [channel,i_brand_id,i_class_id] #18 + CometHashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [channel,i_brand_id,sum,isEmpty,sum] + CometColumnarExchange [channel,i_brand_id] #19 + CometHashAggregate [channel,i_brand_id,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [channel,sum,isEmpty,sum] + CometColumnarExchange [channel] #20 + CometHashAggregate [channel,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [sum,isEmpty,sum] + CometColumnarExchange #21 + CometHashAggregate [sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt index 4d23b269c1..7130612b73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt @@ -1,157 +1,163 @@ == Physical Plan == -TakeOrderedAndProject (153) -+- Union (152) - :- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * ColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan parquet spark_catalog.default.customer (11) - : : : : +- BroadcastExchange (21) - : : : : +- * ColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) - : : : +- BroadcastExchange (27) - : : : +- * ColumnarToRow (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.customer_address (24) - : : +- ReusedExchange (30) - : +- BroadcastExchange (36) - : +- * ColumnarToRow (35) - : +- CometFilter (34) - : +- CometScan parquet spark_catalog.default.item (33) - :- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * Project (53) - : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : :- * Project (50) - : : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : : :- * Project (47) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : : : :- * ColumnarToRow (44) - : : : : : : : +- CometFilter (43) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (42) - : : : : : : +- ReusedExchange (45) - : : : : : +- ReusedExchange (48) - : : : : +- ReusedExchange (51) - : : : +- BroadcastExchange (57) - : : : +- * ColumnarToRow (56) - : : : +- CometFilter (55) - : : : +- CometScan parquet spark_catalog.default.customer_address (54) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - :- * HashAggregate (96) - : +- Exchange (95) - : +- * HashAggregate (94) - : +- * Project (93) - : +- * BroadcastHashJoin Inner BuildRight (92) - : :- * Project (90) - : : +- * BroadcastHashJoin Inner BuildRight (89) - : : :- * Project (87) - : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : :- * Project (80) - : : : : +- * BroadcastHashJoin Inner BuildRight (79) - : : : : :- * Project (77) - : : : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : : : :- * Project (74) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : : : : :- * ColumnarToRow (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (69) - : : : : : : +- ReusedExchange (72) - : : : : : +- ReusedExchange (75) - : : : : +- ReusedExchange (78) - : : : +- BroadcastExchange (85) - : : : +- * ColumnarToRow (84) - : : : +- CometProject (83) - : : : +- CometFilter (82) - : : : +- CometScan parquet spark_catalog.default.customer_address (81) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- * HashAggregate (124) - : +- Exchange (123) - : +- * HashAggregate (122) - : +- * Project (121) - : +- * BroadcastHashJoin Inner BuildRight (120) - : :- * Project (118) - : : +- * BroadcastHashJoin Inner BuildRight (117) - : : :- * Project (115) - : : : +- * BroadcastHashJoin Inner BuildRight (114) - : : : :- * Project (108) - : : : : +- * BroadcastHashJoin Inner BuildRight (107) - : : : : :- * Project (105) - : : : : : +- * BroadcastHashJoin Inner BuildRight (104) - : : : : : :- * Project (102) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (101) - : : : : : : :- * ColumnarToRow (99) - : : : : : : : +- CometFilter (98) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (97) - : : : : : : +- ReusedExchange (100) - : : : : : +- ReusedExchange (103) - : : : : +- ReusedExchange (106) - : : : +- BroadcastExchange (113) - : : : +- * ColumnarToRow (112) - : : : +- CometProject (111) - : : : +- CometFilter (110) - : : : +- CometScan parquet spark_catalog.default.customer_address (109) - : : +- ReusedExchange (116) - : +- ReusedExchange (119) - +- * HashAggregate (151) - +- Exchange (150) - +- * HashAggregate (149) - +- * Project (148) - +- * BroadcastHashJoin Inner BuildRight (147) - :- * Project (142) - : +- * BroadcastHashJoin Inner BuildRight (141) - : :- * Project (139) - : : +- * BroadcastHashJoin Inner BuildRight (138) - : : :- * Project (136) - : : : +- * BroadcastHashJoin Inner BuildRight (135) - : : : :- * Project (133) - : : : : +- * BroadcastHashJoin Inner BuildRight (132) - : : : : :- * Project (130) - : : : : : +- * BroadcastHashJoin Inner BuildRight (129) - : : : : : :- * ColumnarToRow (127) - : : : : : : +- CometFilter (126) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (125) - : : : : : +- ReusedExchange (128) - : : : : +- ReusedExchange (131) - : : : +- ReusedExchange (134) - : : +- ReusedExchange (137) - : +- ReusedExchange (140) - +- BroadcastExchange (146) - +- * ColumnarToRow (145) - +- CometFilter (144) - +- CometScan parquet spark_catalog.default.item (143) +* ColumnarToRow (159) ++- CometTakeOrderedAndProject (158) + +- CometUnion (157) + :- CometHashAggregate (42) + : +- CometColumnarExchange (41) + : +- RowToColumnar (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (32) + : : +- * BroadcastHashJoin Inner BuildRight (31) + : : :- * Project (29) + : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : :- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * ColumnarToRow (7) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- CometProject (13) + : : : : : +- CometFilter (12) + : : : : : +- CometScan parquet spark_catalog.default.customer (11) + : : : : +- BroadcastExchange (21) + : : : : +- * ColumnarToRow (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) + : : : +- BroadcastExchange (27) + : : : +- * ColumnarToRow (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.customer_address (24) + : : +- ReusedExchange (30) + : +- BroadcastExchange (36) + : +- * ColumnarToRow (35) + : +- CometFilter (34) + : +- CometScan parquet spark_catalog.default.item (33) + :- CometHashAggregate (70) + : +- CometColumnarExchange (69) + : +- RowToColumnar (68) + : +- * HashAggregate (67) + : +- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : :- * Project (54) + : : : : +- * BroadcastHashJoin Inner BuildRight (53) + : : : : :- * Project (51) + : : : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : : : :- * Project (48) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : : : :- * ColumnarToRow (45) + : : : : : : : +- CometFilter (44) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (43) + : : : : : : +- ReusedExchange (46) + : : : : : +- ReusedExchange (49) + : : : : +- ReusedExchange (52) + : : : +- BroadcastExchange (58) + : : : +- * ColumnarToRow (57) + : : : +- CometFilter (56) + : : : +- CometScan parquet spark_catalog.default.customer_address (55) + : : +- ReusedExchange (61) + : +- ReusedExchange (64) + :- CometHashAggregate (99) + : +- CometColumnarExchange (98) + : +- RowToColumnar (97) + : +- * HashAggregate (96) + : +- * Project (95) + : +- * BroadcastHashJoin Inner BuildRight (94) + : :- * Project (92) + : : +- * BroadcastHashJoin Inner BuildRight (91) + : : :- * Project (89) + : : : +- * BroadcastHashJoin Inner BuildRight (88) + : : : :- * Project (82) + : : : : +- * BroadcastHashJoin Inner BuildRight (81) + : : : : :- * Project (79) + : : : : : +- * BroadcastHashJoin Inner BuildRight (78) + : : : : : :- * Project (76) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) + : : : : : : :- * ColumnarToRow (73) + : : : : : : : +- CometFilter (72) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (71) + : : : : : : +- ReusedExchange (74) + : : : : : +- ReusedExchange (77) + : : : : +- ReusedExchange (80) + : : : +- BroadcastExchange (87) + : : : +- * ColumnarToRow (86) + : : : +- CometProject (85) + : : : +- CometFilter (84) + : : : +- CometScan parquet spark_catalog.default.customer_address (83) + : : +- ReusedExchange (90) + : +- ReusedExchange (93) + :- CometHashAggregate (128) + : +- CometColumnarExchange (127) + : +- RowToColumnar (126) + : +- * HashAggregate (125) + : +- * Project (124) + : +- * BroadcastHashJoin Inner BuildRight (123) + : :- * Project (121) + : : +- * BroadcastHashJoin Inner BuildRight (120) + : : :- * Project (118) + : : : +- * BroadcastHashJoin Inner BuildRight (117) + : : : :- * Project (111) + : : : : +- * BroadcastHashJoin Inner BuildRight (110) + : : : : :- * Project (108) + : : : : : +- * BroadcastHashJoin Inner BuildRight (107) + : : : : : :- * Project (105) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (104) + : : : : : : :- * ColumnarToRow (102) + : : : : : : : +- CometFilter (101) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (100) + : : : : : : +- ReusedExchange (103) + : : : : : +- ReusedExchange (106) + : : : : +- ReusedExchange (109) + : : : +- BroadcastExchange (116) + : : : +- * ColumnarToRow (115) + : : : +- CometProject (114) + : : : +- CometFilter (113) + : : : +- CometScan parquet spark_catalog.default.customer_address (112) + : : +- ReusedExchange (119) + : +- ReusedExchange (122) + +- CometHashAggregate (156) + +- CometColumnarExchange (155) + +- RowToColumnar (154) + +- * HashAggregate (153) + +- * Project (152) + +- * BroadcastHashJoin Inner BuildRight (151) + :- * Project (146) + : +- * BroadcastHashJoin Inner BuildRight (145) + : :- * Project (143) + : : +- * BroadcastHashJoin Inner BuildRight (142) + : : :- * Project (140) + : : : +- * BroadcastHashJoin Inner BuildRight (139) + : : : :- * Project (137) + : : : : +- * BroadcastHashJoin Inner BuildRight (136) + : : : : :- * Project (134) + : : : : : +- * BroadcastHashJoin Inner BuildRight (133) + : : : : : :- * ColumnarToRow (131) + : : : : : : +- CometFilter (130) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (129) + : : : : : +- ReusedExchange (132) + : : : : +- ReusedExchange (135) + : : : +- ReusedExchange (138) + : : +- ReusedExchange (141) + : +- ReusedExchange (144) + +- BroadcastExchange (150) + +- * ColumnarToRow (149) + +- CometFilter (148) + +- CometScan parquet spark_catalog.default.item (147) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -289,7 +295,7 @@ Join condition: None Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -(30) ReusedExchange [Reuses operator id: 158] +(30) ReusedExchange [Reuses operator id: 164] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 7] @@ -337,68 +343,69 @@ Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30) Aggregate Attributes [14]: [sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48] Results [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -(40) Exchange +(40) RowToColumnar Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, ca_county#22, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) HashAggregate [codegen id : 8] +(41) CometColumnarExchange +Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, ca_county#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(42) CometHashAggregate Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Keys [4]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] -Aggregate Attributes [7]: [avg(agg1#28)#63, avg(agg2#29)#64, avg(agg3#30)#65, avg(agg4#31)#66, avg(agg5#32)#67, avg(agg6#33)#68, avg(agg7#34)#69] -Results [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, avg(agg1#28)#63 AS agg1#70, avg(agg2#29)#64 AS agg2#71, avg(agg3#30)#65 AS agg3#72, avg(agg4#31)#66 AS agg4#73, avg(agg5#32)#67 AS agg5#74, avg(agg6#33)#68 AS agg6#75, avg(agg7#34)#69 AS agg7#76] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#63)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(43) CometFilter +(44) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(44) ColumnarToRow [codegen id : 15] +(45) ColumnarToRow [codegen id : 14] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(45) ReusedExchange [Reuses operator id: 8] +(46) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(46) BroadcastHashJoin [codegen id : 15] +(47) BroadcastHashJoin [codegen id : 14] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(47) Project [codegen id : 15] +(48) Project [codegen id : 14] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(48) ReusedExchange [Reuses operator id: 15] +(49) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(49) BroadcastHashJoin [codegen id : 15] +(50) BroadcastHashJoin [codegen id : 14] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(50) Project [codegen id : 15] +(51) Project [codegen id : 14] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(51) ReusedExchange [Reuses operator id: 21] +(52) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(52) BroadcastHashJoin [codegen id : 15] +(53) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(53) Project [codegen id : 15] +(54) Project [codegen id : 14] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -409,122 +416,123 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(55) CometFilter +(56) CometFilter Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(56) ColumnarToRow [codegen id : 12] +(57) ColumnarToRow [codegen id : 11] Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] -(57) BroadcastExchange +(58) BroadcastExchange Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(58) BroadcastHashJoin [codegen id : 15] +(59) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(59) Project [codegen id : 15] +(60) Project [codegen id : 14] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_state#23, ca_country#24] -(60) ReusedExchange [Reuses operator id: 158] +(61) ReusedExchange [Reuses operator id: 164] Output [1]: [d_date_sk#25] -(61) BroadcastHashJoin [codegen id : 15] +(62) BroadcastHashJoin [codegen id : 14] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(62) Project [codegen id : 15] +(63) Project [codegen id : 14] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24, d_date_sk#25] -(63) ReusedExchange [Reuses operator id: 36] +(64) ReusedExchange [Reuses operator id: 36] Output [2]: [i_item_sk#26, i_item_id#27] -(64) BroadcastHashJoin [codegen id : 15] +(65) BroadcastHashJoin [codegen id : 14] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(65) Project [codegen id : 15] +(66) Project [codegen id : 14] Output [10]: [i_item_id#27, ca_country#24, ca_state#23, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24, i_item_sk#26, i_item_id#27] -(66) HashAggregate [codegen id : 15] +(67) HashAggregate [codegen id : 14] Input [10]: [i_item_id#27, ca_country#24, ca_state#23, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [3]: [i_item_id#27, ca_country#24, ca_state#23] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -Results [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] +Aggregate Attributes [14]: [sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71, sum#72, count#73, sum#74, count#75, sum#76, count#77] +Results [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] + +(68) RowToColumnar +Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -(67) Exchange -Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] -Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(69) CometColumnarExchange +Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] +Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(68) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] +(70) CometHashAggregate +Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] Keys [3]: [i_item_id#27, ca_country#24, ca_state#23] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] -Aggregate Attributes [7]: [avg(agg1#28)#106, avg(agg2#29)#107, avg(agg3#30)#108, avg(agg4#31)#109, avg(agg5#32)#110, avg(agg6#33)#111, avg(agg7#34)#112] -Results [11]: [i_item_id#27, ca_country#24, ca_state#23, null AS county#113, avg(agg1#28)#106 AS agg1#114, avg(agg2#29)#107 AS agg2#115, avg(agg3#30)#108 AS agg3#116, avg(agg4#31)#109 AS agg4#117, avg(agg5#32)#110 AS agg5#118, avg(agg6#33)#111 AS agg6#119, avg(agg7#34)#112 AS agg7#120] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#121)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#92)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(70) CometFilter +(72) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(71) ColumnarToRow [codegen id : 23] +(73) ColumnarToRow [codegen id : 21] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(72) ReusedExchange [Reuses operator id: 8] +(74) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(73) BroadcastHashJoin [codegen id : 23] +(75) BroadcastHashJoin [codegen id : 21] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(74) Project [codegen id : 23] +(76) Project [codegen id : 21] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(75) ReusedExchange [Reuses operator id: 15] +(77) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(76) BroadcastHashJoin [codegen id : 23] +(78) BroadcastHashJoin [codegen id : 21] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(77) Project [codegen id : 23] +(79) Project [codegen id : 21] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(78) ReusedExchange [Reuses operator id: 21] +(80) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(79) BroadcastHashJoin [codegen id : 23] +(81) BroadcastHashJoin [codegen id : 21] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(80) Project [codegen id : 23] +(82) Project [codegen id : 21] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -535,126 +543,127 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(82) CometFilter +(84) CometFilter Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(83) CometProject +(85) CometProject Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_country#24], [ca_address_sk#21, ca_country#24] -(84) ColumnarToRow [codegen id : 20] +(86) ColumnarToRow [codegen id : 18] Input [2]: [ca_address_sk#21, ca_country#24] -(85) BroadcastExchange +(87) BroadcastExchange Input [2]: [ca_address_sk#21, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(86) BroadcastHashJoin [codegen id : 23] +(88) BroadcastHashJoin [codegen id : 21] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(87) Project [codegen id : 23] +(89) Project [codegen id : 21] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_country#24] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_country#24] -(88) ReusedExchange [Reuses operator id: 158] +(90) ReusedExchange [Reuses operator id: 164] Output [1]: [d_date_sk#25] -(89) BroadcastHashJoin [codegen id : 23] +(91) BroadcastHashJoin [codegen id : 21] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(90) Project [codegen id : 23] +(92) Project [codegen id : 21] Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_country#24, d_date_sk#25] -(91) ReusedExchange [Reuses operator id: 36] +(93) ReusedExchange [Reuses operator id: 36] Output [2]: [i_item_sk#26, i_item_id#27] -(92) BroadcastHashJoin [codegen id : 23] +(94) BroadcastHashJoin [codegen id : 21] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(93) Project [codegen id : 23] +(95) Project [codegen id : 21] Output [9]: [i_item_id#27, ca_country#24, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24, i_item_sk#26, i_item_id#27] -(94) HashAggregate [codegen id : 23] +(96) HashAggregate [codegen id : 21] Input [9]: [i_item_id#27, ca_country#24, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [2]: [i_item_id#27, ca_country#24] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] -Results [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] +Aggregate Attributes [14]: [sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104, sum#105, count#106] +Results [16]: [i_item_id#27, ca_country#24, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] + +(97) RowToColumnar +Input [16]: [i_item_id#27, ca_country#24, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] -(95) Exchange -Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] -Arguments: hashpartitioning(i_item_id#27, ca_country#24, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(98) CometColumnarExchange +Input [16]: [i_item_id#27, ca_country#24, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +Arguments: hashpartitioning(i_item_id#27, ca_country#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(96) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] +(99) CometHashAggregate +Input [16]: [i_item_id#27, ca_country#24, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] Keys [2]: [i_item_id#27, ca_country#24] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] -Aggregate Attributes [7]: [avg(agg1#28)#150, avg(agg2#29)#151, avg(agg3#30)#152, avg(agg4#31)#153, avg(agg5#32)#154, avg(agg6#33)#155, avg(agg7#34)#156] -Results [11]: [i_item_id#27, ca_country#24, null AS ca_state#157, null AS county#158, avg(agg1#28)#150 AS agg1#159, avg(agg2#29)#151 AS agg2#160, avg(agg3#30)#152 AS agg3#161, avg(agg4#31)#153 AS agg4#162, avg(agg5#32)#154 AS agg5#163, avg(agg6#33)#155 AS agg6#164, avg(agg7#34)#156 AS agg7#165] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#166)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#121)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(98) CometFilter +(101) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(99) ColumnarToRow [codegen id : 31] +(102) ColumnarToRow [codegen id : 28] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(100) ReusedExchange [Reuses operator id: 8] +(103) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(101) BroadcastHashJoin [codegen id : 31] +(104) BroadcastHashJoin [codegen id : 28] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(102) Project [codegen id : 31] +(105) Project [codegen id : 28] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(103) ReusedExchange [Reuses operator id: 15] +(106) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(104) BroadcastHashJoin [codegen id : 31] +(107) BroadcastHashJoin [codegen id : 28] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(105) Project [codegen id : 31] +(108) Project [codegen id : 28] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(106) ReusedExchange [Reuses operator id: 21] +(109) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(107) BroadcastHashJoin [codegen id : 31] +(110) BroadcastHashJoin [codegen id : 28] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(108) Project [codegen id : 31] +(111) Project [codegen id : 28] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -665,152 +674,153 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(110) CometFilter +(113) CometFilter Input [2]: [ca_address_sk#21, ca_state#23] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(111) CometProject +(114) CometProject Input [2]: [ca_address_sk#21, ca_state#23] Arguments: [ca_address_sk#21], [ca_address_sk#21] -(112) ColumnarToRow [codegen id : 28] +(115) ColumnarToRow [codegen id : 25] Input [1]: [ca_address_sk#21] -(113) BroadcastExchange +(116) BroadcastExchange Input [1]: [ca_address_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(114) BroadcastHashJoin [codegen id : 31] +(117) BroadcastHashJoin [codegen id : 28] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(115) Project [codegen id : 31] +(118) Project [codegen id : 28] Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21] -(116) ReusedExchange [Reuses operator id: 158] +(119) ReusedExchange [Reuses operator id: 164] Output [1]: [d_date_sk#25] -(117) BroadcastHashJoin [codegen id : 31] +(120) BroadcastHashJoin [codegen id : 28] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(118) Project [codegen id : 31] +(121) Project [codegen id : 28] Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, d_date_sk#25] -(119) ReusedExchange [Reuses operator id: 36] +(122) ReusedExchange [Reuses operator id: 36] Output [2]: [i_item_sk#26, i_item_id#27] -(120) BroadcastHashJoin [codegen id : 31] +(123) BroadcastHashJoin [codegen id : 28] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(121) Project [codegen id : 31] +(124) Project [codegen id : 28] Output [8]: [i_item_id#27, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_sk#26, i_item_id#27] -(122) HashAggregate [codegen id : 31] +(125) HashAggregate [codegen id : 28] Input [8]: [i_item_id#27, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [1]: [i_item_id#27] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180] -Results [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] +Aggregate Attributes [14]: [sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Results [15]: [i_item_id#27, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] + +(126) RowToColumnar +Input [15]: [i_item_id#27, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] -(123) Exchange -Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(127) CometColumnarExchange +Input [15]: [i_item_id#27, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] +Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(124) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] +(128) CometHashAggregate +Input [15]: [i_item_id#27, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] Keys [1]: [i_item_id#27] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] -Aggregate Attributes [7]: [avg(agg1#28)#195, avg(agg2#29)#196, avg(agg3#30)#197, avg(agg4#31)#198, avg(agg5#32)#199, avg(agg6#33)#200, avg(agg7#34)#201] -Results [11]: [i_item_id#27, null AS ca_country#202, null AS ca_state#203, null AS county#204, avg(agg1#28)#195 AS agg1#205, avg(agg2#29)#196 AS agg2#206, avg(agg3#30)#197 AS agg3#207, avg(agg4#31)#198 AS agg4#208, avg(agg5#32)#199 AS agg5#209, avg(agg6#33)#200 AS agg6#210, avg(agg7#34)#201 AS agg7#211] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#212)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#150)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(126) CometFilter +(130) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(127) ColumnarToRow [codegen id : 39] +(131) ColumnarToRow [codegen id : 35] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(128) ReusedExchange [Reuses operator id: 8] +(132) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(129) BroadcastHashJoin [codegen id : 39] +(133) BroadcastHashJoin [codegen id : 35] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(130) Project [codegen id : 39] +(134) Project [codegen id : 35] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(131) ReusedExchange [Reuses operator id: 15] +(135) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(132) BroadcastHashJoin [codegen id : 39] +(136) BroadcastHashJoin [codegen id : 35] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(133) Project [codegen id : 39] +(137) Project [codegen id : 35] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(134) ReusedExchange [Reuses operator id: 21] +(138) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(135) BroadcastHashJoin [codegen id : 39] +(139) BroadcastHashJoin [codegen id : 35] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(136) Project [codegen id : 39] +(140) Project [codegen id : 35] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -(137) ReusedExchange [Reuses operator id: 113] +(141) ReusedExchange [Reuses operator id: 116] Output [1]: [ca_address_sk#21] -(138) BroadcastHashJoin [codegen id : 39] +(142) BroadcastHashJoin [codegen id : 35] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(139) Project [codegen id : 39] +(143) Project [codegen id : 35] Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21] -(140) ReusedExchange [Reuses operator id: 158] +(144) ReusedExchange [Reuses operator id: 164] Output [1]: [d_date_sk#25] -(141) BroadcastHashJoin [codegen id : 39] +(145) BroadcastHashJoin [codegen id : 35] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(142) Project [codegen id : 39] +(146) Project [codegen id : 35] Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, d_date_sk#25] @@ -821,89 +831,98 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(144) CometFilter +(148) CometFilter Input [1]: [i_item_sk#26] Condition : isnotnull(i_item_sk#26) -(145) ColumnarToRow [codegen id : 38] +(149) ColumnarToRow [codegen id : 34] Input [1]: [i_item_sk#26] -(146) BroadcastExchange +(150) BroadcastExchange Input [1]: [i_item_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(147) BroadcastHashJoin [codegen id : 39] +(151) BroadcastHashJoin [codegen id : 35] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(148) Project [codegen id : 39] +(152) Project [codegen id : 35] Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_sk#26] -(149) HashAggregate [codegen id : 39] +(153) HashAggregate [codegen id : 35] Input [7]: [agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys: [] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224, sum#225, count#226] -Results [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] +Aggregate Attributes [14]: [sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164] +Results [14]: [sum#165, count#166, sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178] -(150) Exchange -Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] +(154) RowToColumnar +Input [14]: [sum#165, count#166, sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178] -(151) HashAggregate [codegen id : 40] -Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] +(155) CometColumnarExchange +Input [14]: [sum#165, count#166, sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(156) CometHashAggregate +Input [14]: [sum#165, count#166, sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178] Keys: [] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] -Aggregate Attributes [7]: [avg(agg1#28)#241, avg(agg2#29)#242, avg(agg3#30)#243, avg(agg4#31)#244, avg(agg5#32)#245, avg(agg6#33)#246, avg(agg7#34)#247] -Results [11]: [null AS i_item_id#248, null AS ca_country#249, null AS ca_state#250, null AS county#251, avg(agg1#28)#241 AS agg1#252, avg(agg2#29)#242 AS agg2#253, avg(agg3#30)#243 AS agg3#254, avg(agg4#31)#244 AS agg4#255, avg(agg5#32)#245 AS agg5#256, avg(agg6#33)#246 AS agg6#257, avg(agg7#34)#247 AS agg7#258] -(152) Union +(157) CometUnion +Child 0 Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#179, agg2#180, agg3#181, agg4#182, agg5#183, agg6#184, agg7#185] +Child 1 Input [11]: [i_item_id#27, ca_country#24, ca_state#23, county#186, agg1#187, agg2#188, agg3#189, agg4#190, agg5#191, agg6#192, agg7#193] +Child 2 Input [11]: [i_item_id#27, ca_country#24, ca_state#194, county#195, agg1#196, agg2#197, agg3#198, agg4#199, agg5#200, agg6#201, agg7#202] +Child 3 Input [11]: [i_item_id#27, ca_country#203, ca_state#204, county#205, agg1#206, agg2#207, agg3#208, agg4#209, agg5#210, agg6#211, agg7#212] +Child 4 Input [11]: [i_item_id#213, ca_country#214, ca_state#215, county#216, agg1#217, agg2#218, agg3#219, agg4#220, agg5#221, agg6#222, agg7#223] + +(158) CometTakeOrderedAndProject +Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#179, agg2#180, agg3#181, agg4#182, agg5#183, agg6#184, agg7#185] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#24 ASC NULLS FIRST,ca_state#23 ASC NULLS FIRST,ca_county#22 ASC NULLS FIRST,i_item_id#27 ASC NULLS FIRST], output=[i_item_id#27,ca_country#24,ca_state#23,ca_county#22,agg1#179,agg2#180,agg3#181,agg4#182,agg5#183,agg6#184,agg7#185]), 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#27 ASC NULLS FIRST], [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#179, agg2#180, agg3#181, agg4#182, agg5#183, agg6#184, agg7#185] -(153) TakeOrderedAndProject -Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] -Arguments: 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#27 ASC NULLS FIRST], [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] +(159) ColumnarToRow [codegen id : 36] +Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#179, agg2#180, agg3#181, agg4#182, agg5#183, agg6#184, agg7#185] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (158) -+- * ColumnarToRow (157) - +- CometProject (156) - +- CometFilter (155) - +- CometScan parquet spark_catalog.default.date_dim (154) +BroadcastExchange (164) ++- * ColumnarToRow (163) + +- CometProject (162) + +- CometFilter (161) + +- CometScan parquet spark_catalog.default.date_dim (160) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#259] +Output [2]: [d_date_sk#25, d_year#224] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(155) CometFilter -Input [2]: [d_date_sk#25, d_year#259] -Condition : ((isnotnull(d_year#259) AND (d_year#259 = 2001)) AND isnotnull(d_date_sk#25)) +(161) CometFilter +Input [2]: [d_date_sk#25, d_year#224] +Condition : ((isnotnull(d_year#224) AND (d_year#224 = 2001)) AND isnotnull(d_date_sk#25)) -(156) CometProject -Input [2]: [d_date_sk#25, d_year#259] +(162) CometProject +Input [2]: [d_date_sk#25, d_year#224] Arguments: [d_date_sk#25], [d_date_sk#25] -(157) ColumnarToRow [codegen id : 1] +(163) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(158) BroadcastExchange +(164) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:2 Hosting operator id = 42 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 71 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 97 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 100 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index f02809572c..e0eebcd657 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt @@ -1,233 +1,231 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - Union - WholeStageCodegen (8) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id,ca_country,ca_state,ca_county] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +WholeStageCodegen (36) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometUnion + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #6 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (16) - HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id,ca_country,ca_state] #8 - WholeStageCodegen (15) - HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange [i_item_id,ca_country,ca_state] #8 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 InputAdapter - ReusedExchange [cd_demo_sk] #5 + BroadcastExchange #9 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (24) - HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id,ca_country] #10 - WholeStageCodegen (23) - HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #7 + CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange [i_item_id,ca_country] #10 + RowToColumnar + WholeStageCodegen (21) + HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 InputAdapter - ReusedExchange [cd_demo_sk] #5 + BroadcastExchange #11 + WholeStageCodegen (18) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk,ca_country] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (32) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id] #12 - WholeStageCodegen (31) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #7 + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange [i_item_id] #12 + RowToColumnar + WholeStageCodegen (28) + HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 InputAdapter - ReusedExchange [cd_demo_sk] #5 + BroadcastExchange #13 + WholeStageCodegen (25) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (28) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (40) - HashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange #14 - WholeStageCodegen (39) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #7 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange #14 + RowToColumnar + WholeStageCodegen (35) + HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 InputAdapter - ReusedExchange [cd_demo_sk] #5 + ReusedExchange [ca_address_sk] #13 InputAdapter - ReusedExchange [ca_address_sk] #13 + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (38) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] + BroadcastExchange #15 + WholeStageCodegen (34) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index ad52796edc..f62cbd1f86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -64,7 +66,7 @@ Join condition: None Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] +(10) ReusedExchange [Reuses operator id: 27] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -84,66 +86,70 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(14) RowToColumnar Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(15) CometColumnarExchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] -Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16] -(16) Exchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(17) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(18) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] -(17) Sort [codegen id : 5] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [i_class#9 ASC NULLS FIRST], false, 0 +(19) ColumnarToRow [codegen id : 4] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] -(18) Window -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(20) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(19) Project [codegen id : 6] -Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] -Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] +(21) Project [codegen id : 5] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] -(20) TakeOrderedAndProject -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +(22) TakeOrderedAndProject +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +Output [2]: [d_date_sk#11, d_date#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#18] +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#18] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index 2a2a392cd0..4cf648ccf6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -1,16 +1,16 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (5) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - Sort [i_class] + WholeStageCodegen (4) + ColumnarToRow InputAdapter - Exchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometSort [i_class] + CometColumnarExchange [i_class] #1 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt index bdfd6eee0d..9328d2b49d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt @@ -1,26 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * Expand (18) - +- * Project (17) - +- * BroadcastNestedLoopJoin Inner BuildRight (16) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometScan parquet spark_catalog.default.warehouse (13) +* ColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometHashAggregate (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * Expand (18) + +- * Project (17) + +- * BroadcastNestedLoopJoin Inner BuildRight (16) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometScan parquet spark_catalog.default.warehouse (13) (unknown) Scan parquet spark_catalog.default.inventory @@ -38,7 +40,7 @@ Condition : isnotnull(inv_item_sk#1) (3) ColumnarToRow [codegen id : 4] Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -(4) ReusedExchange [Reuses operator id: 27] +(4) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 4] @@ -111,50 +113,54 @@ Functions [1]: [partial_avg(inv_quantity_on_hand#2)] Aggregate Attributes [2]: [sum#16, count#17] Results [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] -(20) Exchange +(20) RowToColumnar Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) HashAggregate [codegen id : 5] +(21) CometColumnarExchange +Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] +Arguments: hashpartitioning(i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) CometHashAggregate Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] Keys [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#20] -Results [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, avg(inv_quantity_on_hand#2)#20 AS qoh#21] -(22) TakeOrderedAndProject -Input [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] -Arguments: 100, [qoh#21 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#12 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_category#14 ASC NULLS FIRST], [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] +(23) CometTakeOrderedAndProject +Input [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#20] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#20 ASC NULLS FIRST,i_product_name#11 ASC NULLS FIRST,i_brand#12 ASC NULLS FIRST,i_class#13 ASC NULLS FIRST,i_category#14 ASC NULLS FIRST], output=[i_product_name#11,i_brand#12,i_class#13,i_category#14,qoh#20]), 100, [qoh#20 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#12 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_category#14 ASC NULLS FIRST], [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#20] + +(24) ColumnarToRow [codegen id : 5] +Input [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#22] +Output [2]: [d_date_sk#5, d_month_seq#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#22] -Condition : (((isnotnull(d_month_seq#22) AND (d_month_seq#22 >= 1200)) AND (d_month_seq#22 <= 1211)) AND isnotnull(d_date_sk#5)) +(26) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#21] +Condition : (((isnotnull(d_month_seq#21) AND (d_month_seq#21 >= 1200)) AND (d_month_seq#21 <= 1211)) AND isnotnull(d_date_sk#5)) -(25) CometProject -Input [2]: [d_date_sk#5, d_month_seq#22] +(27) CometProject +Input [2]: [d_date_sk#5, d_month_seq#21] Arguments: [d_date_sk#5], [d_date_sk#5] -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 63a428d4e4..797b870040 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -1,41 +1,43 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - InputAdapter - Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + BroadcastNestedLoopJoin + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.warehouse + CometScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index e0a290cea7..3911ebef4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- Union (44) - :- * HashAggregate (23) - : +- * HashAggregate (22) - : +- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.warehouse (13) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * HashAggregate (25) - : +- ReusedExchange (24) - :- * HashAggregate (33) - : +- Exchange (32) - : +- * HashAggregate (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) - :- * HashAggregate (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- * HashAggregate (35) - : +- ReusedExchange (34) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * HashAggregate (40) - +- ReusedExchange (39) +* ColumnarToRow (47) ++- CometTakeOrderedAndProject (46) + +- CometUnion (45) + :- CometHashAggregate (24) + : +- CometHashAggregate (23) + : +- CometHashAggregate (22) + : +- CometColumnarExchange (21) + : +- RowToColumnar (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.item (7) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.warehouse (13) + :- CometHashAggregate (29) + : +- CometColumnarExchange (28) + : +- CometHashAggregate (27) + : +- CometHashAggregate (26) + : +- ReusedExchange (25) + :- CometHashAggregate (34) + : +- CometColumnarExchange (33) + : +- CometHashAggregate (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + :- CometHashAggregate (39) + : +- CometColumnarExchange (38) + : +- CometHashAggregate (37) + : +- CometHashAggregate (36) + : +- ReusedExchange (35) + +- CometHashAggregate (44) + +- CometColumnarExchange (43) + +- CometHashAggregate (42) + +- CometHashAggregate (41) + +- ReusedExchange (40) (unknown) Scan parquet spark_catalog.default.inventory @@ -61,7 +63,7 @@ Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (3) ColumnarToRow [codegen id : 4] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(4) ReusedExchange [Reuses operator id: 50] +(4) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 4] @@ -137,178 +139,159 @@ Functions [1]: [partial_avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [sum#13, count#14] Results [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(20) Exchange +(20) RowToColumnar Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, i_category#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) HashAggregate [codegen id : 5] +(21) CometColumnarExchange +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] +Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) CometHashAggregate Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(22) HashAggregate [codegen id : 5] -Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#18] +(23) CometHashAggregate +Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#17] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] -Functions [1]: [partial_avg(qoh#18)] -Aggregate Attributes [2]: [sum#19, count#20] -Results [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] +Functions [1]: [partial_avg(qoh#17)] -(23) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] +(24) CometHashAggregate +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#18, count#19] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] -Functions [1]: [avg(qoh#18)] -Aggregate Attributes [1]: [avg(qoh#18)#23] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(qoh#18)#23 AS qoh#24] +Functions [1]: [avg(qoh#17)] -(24) ReusedExchange [Reuses operator id: 20] +(25) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(25) HashAggregate [codegen id : 10] +(26) CometHashAggregate Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] -Results [4]: [i_product_name#11, i_brand#8, i_class#9, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(26) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#11, i_brand#8, i_class#9, qoh#18] +(27) CometHashAggregate +Input [4]: [i_product_name#11, i_brand#8, i_class#9, qoh#17] Keys [3]: [i_product_name#11, i_brand#8, i_class#9] -Functions [1]: [partial_avg(qoh#18)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] +Functions [1]: [partial_avg(qoh#17)] -(27) Exchange -Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] -Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(28) CometColumnarExchange +Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#20, count#21] +Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] +(29) CometHashAggregate +Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#20, count#21] Keys [3]: [i_product_name#11, i_brand#8, i_class#9] -Functions [1]: [avg(qoh#18)] -Aggregate Attributes [1]: [avg(qoh#18)#29] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, null AS i_category#30, avg(qoh#18)#29 AS qoh#31] +Functions [1]: [avg(qoh#17)] -(29) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(30) HashAggregate [codegen id : 16] +(31) CometHashAggregate Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] -Results [3]: [i_product_name#11, i_brand#8, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(31) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#11, i_brand#8, qoh#18] +(32) CometHashAggregate +Input [3]: [i_product_name#11, i_brand#8, qoh#17] Keys [2]: [i_product_name#11, i_brand#8] -Functions [1]: [partial_avg(qoh#18)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [4]: [i_product_name#11, i_brand#8, sum#34, count#35] +Functions [1]: [partial_avg(qoh#17)] -(32) Exchange -Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] -Arguments: hashpartitioning(i_product_name#11, i_brand#8, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(33) CometColumnarExchange +Input [4]: [i_product_name#11, i_brand#8, sum#22, count#23] +Arguments: hashpartitioning(i_product_name#11, i_brand#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(33) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] +(34) CometHashAggregate +Input [4]: [i_product_name#11, i_brand#8, sum#22, count#23] Keys [2]: [i_product_name#11, i_brand#8] -Functions [1]: [avg(qoh#18)] -Aggregate Attributes [1]: [avg(qoh#18)#36] -Results [5]: [i_product_name#11, i_brand#8, null AS i_class#37, null AS i_category#38, avg(qoh#18)#36 AS qoh#39] +Functions [1]: [avg(qoh#17)] -(34) ReusedExchange [Reuses operator id: 20] +(35) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(35) HashAggregate [codegen id : 22] +(36) CometHashAggregate Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] -Results [2]: [i_product_name#11, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(36) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#11, qoh#18] +(37) CometHashAggregate +Input [2]: [i_product_name#11, qoh#17] Keys [1]: [i_product_name#11] -Functions [1]: [partial_avg(qoh#18)] -Aggregate Attributes [2]: [sum#40, count#41] -Results [3]: [i_product_name#11, sum#42, count#43] +Functions [1]: [partial_avg(qoh#17)] -(37) Exchange -Input [3]: [i_product_name#11, sum#42, count#43] -Arguments: hashpartitioning(i_product_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(38) CometColumnarExchange +Input [3]: [i_product_name#11, sum#24, count#25] +Arguments: hashpartitioning(i_product_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(38) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#11, sum#42, count#43] +(39) CometHashAggregate +Input [3]: [i_product_name#11, sum#24, count#25] Keys [1]: [i_product_name#11] -Functions [1]: [avg(qoh#18)] -Aggregate Attributes [1]: [avg(qoh#18)#44] -Results [5]: [i_product_name#11, null AS i_brand#45, null AS i_class#46, null AS i_category#47, avg(qoh#18)#44 AS qoh#48] +Functions [1]: [avg(qoh#17)] -(39) ReusedExchange [Reuses operator id: 20] +(40) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(40) HashAggregate [codegen id : 28] +(41) CometHashAggregate Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] -Results [1]: [avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(41) HashAggregate [codegen id : 28] -Input [1]: [qoh#18] +(42) CometHashAggregate +Input [1]: [qoh#17] Keys: [] -Functions [1]: [partial_avg(qoh#18)] -Aggregate Attributes [2]: [sum#49, count#50] -Results [2]: [sum#51, count#52] +Functions [1]: [partial_avg(qoh#17)] -(42) Exchange -Input [2]: [sum#51, count#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(43) CometColumnarExchange +Input [2]: [sum#26, count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(43) HashAggregate [codegen id : 29] -Input [2]: [sum#51, count#52] +(44) CometHashAggregate +Input [2]: [sum#26, count#27] Keys: [] -Functions [1]: [avg(qoh#18)] -Aggregate Attributes [1]: [avg(qoh#18)#53] -Results [5]: [null AS i_product_name#54, null AS i_brand#55, null AS i_class#56, null AS i_category#57, avg(qoh#18)#53 AS qoh#58] +Functions [1]: [avg(qoh#17)] + +(45) CometUnion +Child 0 Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#28] +Child 1 Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#29, qoh#30] +Child 2 Input [5]: [i_product_name#11, i_brand#8, i_class#31, i_category#32, qoh#33] +Child 3 Input [5]: [i_product_name#11, i_brand#34, i_class#35, i_category#36, qoh#37] +Child 4 Input [5]: [i_product_name#38, i_brand#39, i_class#40, i_category#41, qoh#42] -(44) Union +(46) CometTakeOrderedAndProject +Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#28 ASC NULLS FIRST,i_product_name#11 ASC NULLS FIRST,i_brand#8 ASC NULLS FIRST,i_class#9 ASC NULLS FIRST,i_category#10 ASC NULLS FIRST], output=[i_product_name#11,i_brand#8,i_class#9,i_category#10,qoh#28]), 100, [qoh#28 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#28] -(45) TakeOrderedAndProject -Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] -Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] +(47) ColumnarToRow [codegen id : 21] +Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#59] +Output [2]: [d_date_sk#6, d_month_seq#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#59] -Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1212)) AND (d_month_seq#59 <= 1223)) AND isnotnull(d_date_sk#6)) +(49) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#43] +Condition : (((isnotnull(d_month_seq#43) AND (d_month_seq#43 >= 1212)) AND (d_month_seq#43 <= 1223)) AND isnotnull(d_date_sk#6)) -(48) CometProject -Input [2]: [d_date_sk#6, d_month_seq#59] +(50) CometProject +Input [2]: [d_date_sk#6, d_month_seq#43] Arguments: [d_date_sk#6], [d_date_sk#6] -(49) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index a8d71b06ac..442e169eb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -1,80 +1,66 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - Union - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - InputAdapter - Exchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +WholeStageCodegen (21) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] + CometUnion + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] + CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] - WholeStageCodegen (11) - HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] - InputAdapter - Exchange [i_product_name,i_brand,i_class] #5 - WholeStageCodegen (10) - HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (17) - HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] - InputAdapter - Exchange [i_product_name,i_brand] #6 - WholeStageCodegen (16) - HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (23) - HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - InputAdapter - Exchange [i_product_name] #7 - WholeStageCodegen (22) - HashAggregate [i_product_name,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (29) - HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - InputAdapter - Exchange #8 - WholeStageCodegen (28) - HashAggregate [qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometHashAggregate [i_product_name,i_brand,i_class,sum,count] + CometColumnarExchange [i_product_name,i_brand,i_class] #5 + CometHashAggregate [i_product_name,i_brand,i_class,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [i_product_name,i_brand,sum,count] + CometColumnarExchange [i_product_name,i_brand] #6 + CometHashAggregate [i_product_name,i_brand,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [i_product_name,sum,count] + CometColumnarExchange [i_product_name] #7 + CometHashAggregate [i_product_name,qoh] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [sum,count] + CometColumnarExchange #8 + CometHashAggregate [qoh] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index 9bbf5e9847..c19e023683 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -1,51 +1,53 @@ == Physical Plan == -* Sort (47) -+- Exchange (46) - +- * Filter (45) - +- * HashAggregate (44) - +- Exchange (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.item (21) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_address (33) +* ColumnarToRow (49) ++- CometSort (48) + +- CometColumnarExchange (47) + +- CometFilter (46) + +- CometHashAggregate (45) + +- CometColumnarExchange (44) + +- CometHashAggregate (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * ColumnarToRow (13) + : : : : +- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.store (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.item (21) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_address (33) (unknown) Scan parquet spark_catalog.default.store_sales @@ -63,9 +65,9 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) CometExchange +(4) CometColumnarExchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] (5) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] @@ -86,9 +88,9 @@ Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(9) CometExchange +(9) CometColumnarExchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (10) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] @@ -229,116 +231,118 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#30] Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] -(40) Exchange +(40) RowToColumnar Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(41) HashAggregate [codegen id : 6] +(41) CometColumnarExchange +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(42) CometHashAggregate Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] -Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] -(42) HashAggregate [codegen id : 6] -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, netpaid#33] +(43) CometHashAggregate +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, netpaid#32] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] -Functions [1]: [partial_sum(netpaid#33)] -Aggregate Attributes [2]: [sum#34, isEmpty#35] -Results [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] +Functions [1]: [partial_sum(netpaid#32)] -(43) Exchange -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(44) CometColumnarExchange +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#33, isEmpty#34] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(44) HashAggregate [codegen id : 7] -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] +(45) CometHashAggregate +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#33, isEmpty#34] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] -Functions [1]: [sum(netpaid#33)] -Aggregate Attributes [1]: [sum(netpaid#33)#38] -Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, sum(netpaid#33)#38 AS paid#39] +Functions [1]: [sum(netpaid#32)] -(45) Filter [codegen id : 7] -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -Condition : (isnotnull(paid#39) AND (cast(paid#39 as decimal(33,8)) > cast(Subquery scalar-subquery#40, [id=#41] as decimal(33,8)))) +(46) CometFilter +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#35] +Condition : (isnotnull(paid#35) AND (cast(paid#35 as decimal(33,8)) > cast(Subquery scalar-subquery#36, [id=#37] as decimal(33,8)))) -(46) Exchange -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(47) CometColumnarExchange +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#35] +Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(47) Sort [codegen id : 8] -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -Arguments: [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], true, 0 +(48) CometSort +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#35] +Arguments: [c_last_name#24, c_first_name#23, s_store_name#11, paid#35], [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] + +(49) ColumnarToRow [codegen id : 6] +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#35] ===== Subqueries ===== -Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* HashAggregate (75) -+- Exchange (74) - +- * HashAggregate (73) - +- * HashAggregate (72) - +- Exchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : :- * ColumnarToRow (54) - : : : : +- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (55) - : : +- BroadcastExchange (61) - : : +- * ColumnarToRow (60) - : : +- CometFilter (59) - : : +- CometScan parquet spark_catalog.default.item (58) - : +- ReusedExchange (64) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#36, [id=#37] +* ColumnarToRow (79) ++- CometHashAggregate (78) + +- CometColumnarExchange (77) + +- CometHashAggregate (76) + +- CometHashAggregate (75) + +- CometColumnarExchange (74) + +- RowToColumnar (73) + +- * HashAggregate (72) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * Project (65) + : : +- * BroadcastHashJoin Inner BuildRight (64) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * ColumnarToRow (56) + : : : : +- CometProject (55) + : : : : +- CometSortMergeJoin (54) + : : : : :- CometSort (51) + : : : : : +- ReusedExchange (50) + : : : : +- CometSort (53) + : : : : +- ReusedExchange (52) + : : : +- ReusedExchange (57) + : : +- BroadcastExchange (63) + : : +- * ColumnarToRow (62) + : : +- CometFilter (61) + : : +- CometScan parquet spark_catalog.default.item (60) + : +- ReusedExchange (66) + +- ReusedExchange (69) + + +(50) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(49) CometSort +(51) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(50) ReusedExchange [Reuses operator id: 9] +(52) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(51) CometSort +(53) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(52) CometSortMergeJoin +(54) CometSortMergeJoin Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Right output [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(53) CometProject +(55) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(54) ColumnarToRow [codegen id : 5] +(56) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(55) ReusedExchange [Reuses operator id: 18] +(57) ReusedExchange [Reuses operator id: 18] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(56) BroadcastHashJoin [codegen id : 5] +(58) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(57) Project [codegen id : 5] +(59) Project [codegen id : 5] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -349,87 +353,87 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) CometFilter +(61) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : isnotnull(i_item_sk#15) -(60) ColumnarToRow [codegen id : 2] +(62) ColumnarToRow [codegen id : 2] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(61) BroadcastExchange +(63) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(62) BroadcastHashJoin [codegen id : 5] +(64) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(63) Project [codegen id : 5] +(65) Project [codegen id : 5] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(64) ReusedExchange [Reuses operator id: 30] +(66) ReusedExchange [Reuses operator id: 30] Output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(65) BroadcastHashJoin [codegen id : 5] +(67) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(66) Project [codegen id : 5] +(68) Project [codegen id : 5] Output [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(67) ReusedExchange [Reuses operator id: 36] +(69) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(68) BroadcastHashJoin [codegen id : 5] +(70) BroadcastHashJoin [codegen id : 5] Left keys [3]: [c_current_addr_sk#22, c_birth_country#25, s_zip#14] Right keys [3]: [ca_address_sk#26, upper(ca_country#29), ca_zip#28] Join type: Inner Join condition: None -(69) Project [codegen id : 5] +(71) Project [codegen id : 5] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25, ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(70) HashAggregate [codegen id : 5] +(72) HashAggregate [codegen id : 5] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#42] -Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] +Aggregate Attributes [1]: [sum#38] +Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#39] + +(73) RowToColumnar +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#39] -(71) Exchange -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(74) CometColumnarExchange +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#39] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(72) HashAggregate [codegen id : 6] -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] +(75) CometHashAggregate +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#39] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] -(73) HashAggregate [codegen id : 6] -Input [1]: [netpaid#33] +(76) CometHashAggregate +Input [1]: [netpaid#32] Keys: [] -Functions [1]: [partial_avg(netpaid#33)] -Aggregate Attributes [2]: [sum#44, count#45] -Results [2]: [sum#46, count#47] +Functions [1]: [partial_avg(netpaid#32)] -(74) Exchange -Input [2]: [sum#46, count#47] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(77) CometColumnarExchange +Input [2]: [sum#40, count#41] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(75) HashAggregate [codegen id : 7] -Input [2]: [sum#46, count#47] +(78) CometHashAggregate +Input [2]: [sum#40, count#41] Keys: [] -Functions [1]: [avg(netpaid#33)] -Aggregate Attributes [1]: [avg(netpaid#33)#48] -Results [1]: [(0.05 * avg(netpaid#33)#48) AS (0.05 * avg(netpaid))#49] +Functions [1]: [avg(netpaid#32)] + +(79) ColumnarToRow [codegen id : 6] +Input [1]: [(0.05 * avg(netpaid))#42] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index b22891cebe..05de849be4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -1,19 +1,19 @@ -WholeStageCodegen (8) - Sort [c_last_name,c_first_name,s_store_name] +WholeStageCodegen (6) + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (7) - Filter [paid] + CometSort [c_last_name,c_first_name,s_store_name] + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + CometFilter [paid] Subquery #1 - WholeStageCodegen (7) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + WholeStageCodegen (6) + ColumnarToRow InputAdapter - Exchange #10 - WholeStageCodegen (6) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + CometHashAggregate [sum,count] + CometColumnarExchange #10 + CometHashAggregate [netpaid] + CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + RowToColumnar WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] @@ -45,64 +45,62 @@ WholeStageCodegen (8) ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometSort [sr_ticket_number,sr_item_sk] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometHashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 + CometHashAggregate [c_last_name,c_first_name,s_store_name,netpaid] + CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #4 + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometSort [sr_ticket_number,sr_item_sk] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #5 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) + BroadcastExchange #6 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) + BroadcastExchange #7 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometFilter [i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) + BroadcastExchange #8 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_country,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt index 54aadf2cf5..d689cb65a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt @@ -1,77 +1,81 @@ == Physical Plan == -TakeOrderedAndProject (73) -+- Union (72) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.item (20) - :- * HashAggregate (50) - : +- Exchange (49) - : +- * HashAggregate (48) - : +- * Project (47) - : +- * BroadcastHashJoin Inner BuildRight (46) - : :- * Project (44) - : : +- * BroadcastHashJoin Inner BuildRight (43) - : : :- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * ColumnarToRow (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) - : : : : +- ReusedExchange (32) - : : : +- ReusedExchange (35) - : : +- BroadcastExchange (42) - : : +- * ColumnarToRow (41) - : : +- CometProject (40) - : : +- CometFilter (39) - : : +- CometScan parquet spark_catalog.default.store (38) - : +- ReusedExchange (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Project (56) - : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : :- * ColumnarToRow (53) - : : : : +- CometFilter (52) - : : : : +- CometScan parquet spark_catalog.default.store_sales (51) - : : : +- ReusedExchange (54) - : : +- ReusedExchange (57) - : +- ReusedExchange (60) - +- BroadcastExchange (66) - +- * ColumnarToRow (65) - +- CometFilter (64) - +- CometScan parquet spark_catalog.default.item (63) +* ColumnarToRow (77) ++- CometTakeOrderedAndProject (76) + +- CometUnion (75) + :- CometHashAggregate (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (17) + : : +- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.store (14) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + :- CometHashAggregate (52) + : +- CometColumnarExchange (51) + : +- RowToColumnar (50) + : +- * HashAggregate (49) + : +- * Project (48) + : +- * BroadcastHashJoin Inner BuildRight (47) + : :- * Project (45) + : : +- * BroadcastHashJoin Inner BuildRight (44) + : : :- * Project (38) + : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : :- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * ColumnarToRow (32) + : : : : : +- CometFilter (31) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (30) + : : : : +- ReusedExchange (33) + : : : +- ReusedExchange (36) + : : +- BroadcastExchange (43) + : : +- * ColumnarToRow (42) + : : +- CometProject (41) + : : +- CometFilter (40) + : : +- CometScan parquet spark_catalog.default.store (39) + : +- ReusedExchange (46) + +- CometHashAggregate (74) + +- CometColumnarExchange (73) + +- RowToColumnar (72) + +- * HashAggregate (71) + +- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * Project (58) + : : : +- * BroadcastHashJoin Inner BuildRight (57) + : : : :- * ColumnarToRow (55) + : : : : +- CometFilter (54) + : : : : +- CometScan parquet spark_catalog.default.store_sales (53) + : : : +- ReusedExchange (56) + : : +- ReusedExchange (59) + : +- ReusedExchange (62) + +- BroadcastExchange (68) + +- * ColumnarToRow (67) + +- CometFilter (66) + +- CometScan parquet spark_catalog.default.item (65) (unknown) Scan parquet spark_catalog.default.store_sales @@ -121,7 +125,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(11) ReusedExchange [Reuses operator id: 78] +(11) ReusedExchange [Reuses operator id: 82] Output [1]: [d_date_sk#14] (12) BroadcastHashJoin [codegen id : 5] @@ -197,55 +201,56 @@ Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), parti Aggregate Attributes [8]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30] Results [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] -(27) Exchange +(27) RowToColumnar Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] -Arguments: hashpartitioning(i_item_id#18, s_state#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) HashAggregate [codegen id : 6] +(28) CometColumnarExchange +Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] +Arguments: hashpartitioning(i_item_id#18, s_state#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(29) CometHashAggregate Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] Keys [2]: [i_item_id#18, s_state#16] Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] -Aggregate Attributes [4]: [avg(agg1#19)#39, avg(UnscaledValue(agg2#20))#40, avg(UnscaledValue(agg3#21))#41, avg(UnscaledValue(agg4#22))#42] -Results [7]: [i_item_id#18, s_state#16, 0 AS g_state#43, avg(agg1#19)#39 AS agg1#44, cast((avg(UnscaledValue(agg2#20))#40 / 100.0) as decimal(11,6)) AS agg2#45, cast((avg(UnscaledValue(agg3#21))#41 / 100.0) as decimal(11,6)) AS agg3#46, cast((avg(UnscaledValue(agg4#22))#42 / 100.0) as decimal(11,6)) AS agg4#47] (unknown) Scan parquet spark_catalog.default.store_sales Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#48)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#39)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(30) CometFilter +(31) CometFilter Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) -(31) ColumnarToRow [codegen id : 11] +(32) ColumnarToRow [codegen id : 10] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(32) ReusedExchange [Reuses operator id: 8] +(33) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#10] -(33) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(34) Project [codegen id : 11] +(35) Project [codegen id : 10] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(35) ReusedExchange [Reuses operator id: 78] +(36) ReusedExchange [Reuses operator id: 82] Output [1]: [d_date_sk#14] -(36) BroadcastHashJoin [codegen id : 11] +(37) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(37) Project [codegen id : 11] +(38) Project [codegen id : 10] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -256,113 +261,114 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(39) CometFilter +(40) CometFilter Input [2]: [s_store_sk#15, s_state#16] Condition : ((isnotnull(s_state#16) AND (s_state#16 = TN)) AND isnotnull(s_store_sk#15)) -(40) CometProject +(41) CometProject Input [2]: [s_store_sk#15, s_state#16] Arguments: [s_store_sk#15], [s_store_sk#15] -(41) ColumnarToRow [codegen id : 9] +(42) ColumnarToRow [codegen id : 8] Input [1]: [s_store_sk#15] -(42) BroadcastExchange +(43) BroadcastExchange Input [1]: [s_store_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 11] +(44) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(44) Project [codegen id : 11] +(45) Project [codegen id : 10] Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15] -(45) ReusedExchange [Reuses operator id: 23] +(46) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#17, i_item_id#18] -(46) BroadcastHashJoin [codegen id : 11] +(47) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(47) Project [codegen id : 11] +(48) Project [codegen id : 10] Output [5]: [i_item_id#18, ss_quantity#4 AS agg1#19, ss_list_price#5 AS agg2#20, ss_coupon_amt#7 AS agg3#21, ss_sales_price#6 AS agg4#22] Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#17, i_item_id#18] -(48) HashAggregate [codegen id : 11] +(49) HashAggregate [codegen id : 10] Input [5]: [i_item_id#18, agg1#19, agg2#20, agg3#21, agg4#22] Keys [1]: [i_item_id#18] Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), partial_avg(UnscaledValue(agg3#21)), partial_avg(UnscaledValue(agg4#22))] -Aggregate Attributes [8]: [sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56] -Results [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] +Aggregate Attributes [8]: [sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47] +Results [9]: [i_item_id#18, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] + +(50) RowToColumnar +Input [9]: [i_item_id#18, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -(49) Exchange -Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(51) CometColumnarExchange +Input [9]: [i_item_id#18, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(50) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] +(52) CometHashAggregate +Input [9]: [i_item_id#18, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] Keys [1]: [i_item_id#18] Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] -Aggregate Attributes [4]: [avg(agg1#19)#65, avg(UnscaledValue(agg2#20))#66, avg(UnscaledValue(agg3#21))#67, avg(UnscaledValue(agg4#22))#68] -Results [7]: [i_item_id#18, null AS s_state#69, 1 AS g_state#70, avg(agg1#19)#65 AS agg1#71, cast((avg(UnscaledValue(agg2#20))#66 / 100.0) as decimal(11,6)) AS agg2#72, cast((avg(UnscaledValue(agg3#21))#67 / 100.0) as decimal(11,6)) AS agg3#73, cast((avg(UnscaledValue(agg4#22))#68 / 100.0) as decimal(11,6)) AS agg4#74] (unknown) Scan parquet spark_catalog.default.store_sales Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#56)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(52) CometFilter +(54) CometFilter Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) -(53) ColumnarToRow [codegen id : 17] +(55) ColumnarToRow [codegen id : 15] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(54) ReusedExchange [Reuses operator id: 8] +(56) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#10] -(55) BroadcastHashJoin [codegen id : 17] +(57) BroadcastHashJoin [codegen id : 15] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(56) Project [codegen id : 17] +(58) Project [codegen id : 15] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(57) ReusedExchange [Reuses operator id: 78] +(59) ReusedExchange [Reuses operator id: 82] Output [1]: [d_date_sk#14] -(58) BroadcastHashJoin [codegen id : 17] +(60) BroadcastHashJoin [codegen id : 15] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(61) Project [codegen id : 15] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -(60) ReusedExchange [Reuses operator id: 42] +(62) ReusedExchange [Reuses operator id: 43] Output [1]: [s_store_sk#15] -(61) BroadcastHashJoin [codegen id : 17] +(63) BroadcastHashJoin [codegen id : 15] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(62) Project [codegen id : 17] +(64) Project [codegen id : 15] Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15] @@ -373,85 +379,92 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(64) CometFilter +(66) CometFilter Input [1]: [i_item_sk#17] Condition : isnotnull(i_item_sk#17) -(65) ColumnarToRow [codegen id : 16] +(67) ColumnarToRow [codegen id : 14] Input [1]: [i_item_sk#17] -(66) BroadcastExchange +(68) BroadcastExchange Input [1]: [i_item_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(67) BroadcastHashJoin [codegen id : 17] +(69) BroadcastHashJoin [codegen id : 15] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(68) Project [codegen id : 17] +(70) Project [codegen id : 15] Output [4]: [ss_quantity#4 AS agg1#19, ss_list_price#5 AS agg2#20, ss_coupon_amt#7 AS agg3#21, ss_sales_price#6 AS agg4#22] Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#17] -(69) HashAggregate [codegen id : 17] +(71) HashAggregate [codegen id : 15] Input [4]: [agg1#19, agg2#20, agg3#21, agg4#22] Keys: [] Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), partial_avg(UnscaledValue(agg3#21)), partial_avg(UnscaledValue(agg4#22))] -Aggregate Attributes [8]: [sum#76, count#77, sum#78, count#79, sum#80, count#81, sum#82, count#83] -Results [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] +Aggregate Attributes [8]: [sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] +Results [8]: [sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72] -(70) Exchange -Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] +(72) RowToColumnar +Input [8]: [sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72] -(71) HashAggregate [codegen id : 18] -Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] +(73) CometColumnarExchange +Input [8]: [sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(74) CometHashAggregate +Input [8]: [sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72] Keys: [] Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] -Aggregate Attributes [4]: [avg(agg1#19)#92, avg(UnscaledValue(agg2#20))#93, avg(UnscaledValue(agg3#21))#94, avg(UnscaledValue(agg4#22))#95] -Results [7]: [null AS i_item_id#96, null AS s_state#97, 1 AS g_state#98, avg(agg1#19)#92 AS agg1#99, cast((avg(UnscaledValue(agg2#20))#93 / 100.0) as decimal(11,6)) AS agg2#100, cast((avg(UnscaledValue(agg3#21))#94 / 100.0) as decimal(11,6)) AS agg3#101, cast((avg(UnscaledValue(agg4#22))#95 / 100.0) as decimal(11,6)) AS agg4#102] -(72) Union +(75) CometUnion +Child 0 Input [7]: [i_item_id#18, s_state#16, g_state#73, agg1#74, agg2#75, agg3#76, agg4#77] +Child 1 Input [7]: [i_item_id#18, s_state#78, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83] +Child 2 Input [7]: [i_item_id#84, s_state#85, g_state#86, agg1#87, agg2#88, agg3#89, agg4#90] + +(76) CometTakeOrderedAndProject +Input [7]: [i_item_id#18, s_state#16, g_state#73, agg1#74, agg2#75, agg3#76, agg4#77] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST,s_state#16 ASC NULLS FIRST], output=[i_item_id#18,s_state#16,g_state#73,agg1#74,agg2#75,agg3#76,agg4#77]), 100, [i_item_id#18 ASC NULLS FIRST, s_state#16 ASC NULLS FIRST], [i_item_id#18, s_state#16, g_state#73, agg1#74, agg2#75, agg3#76, agg4#77] -(73) TakeOrderedAndProject -Input [7]: [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] -Arguments: 100, [i_item_id#18 ASC NULLS FIRST, s_state#16 ASC NULLS FIRST], [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] +(77) ColumnarToRow [codegen id : 16] +Input [7]: [i_item_id#18, s_state#16, g_state#73, agg1#74, agg2#75, agg3#76, agg4#77] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (78) -+- * ColumnarToRow (77) - +- CometProject (76) - +- CometFilter (75) - +- CometScan parquet spark_catalog.default.date_dim (74) +BroadcastExchange (82) ++- * ColumnarToRow (81) + +- CometProject (80) + +- CometFilter (79) + +- CometScan parquet spark_catalog.default.date_dim (78) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#103] +Output [2]: [d_date_sk#14, d_year#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(75) CometFilter -Input [2]: [d_date_sk#14, d_year#103] -Condition : ((isnotnull(d_year#103) AND (d_year#103 = 1998)) AND isnotnull(d_date_sk#14)) +(79) CometFilter +Input [2]: [d_date_sk#14, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 1998)) AND isnotnull(d_date_sk#14)) -(76) CometProject -Input [2]: [d_date_sk#14, d_year#103] +(80) CometProject +Input [2]: [d_date_sk#14, d_year#91] Arguments: [d_date_sk#14], [d_date_sk#14] -(77) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(78) BroadcastExchange +(82) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt index 32f003798d..fec1c27dd3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt @@ -1,117 +1,117 @@ -TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id,s_state] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + CometUnion + CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange [i_item_id,s_state] #1 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange [i_item_id] #6 - WholeStageCodegen (11) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange [i_item_id] #6 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk] #3 InputAdapter - ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - InputAdapter - Exchange #8 - WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #5 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] + CometColumnarExchange #8 + RowToColumnar + WholeStageCodegen (15) + HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk] #3 InputAdapter - ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [s_store_sk] #7 InputAdapter - ReusedExchange [s_store_sk] #7 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] + BroadcastExchange #9 + WholeStageCodegen (14) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt index a86edcfa3a..cd5187ddd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt @@ -1,36 +1,40 @@ == Physical Plan == -* Sort (32) -+- Exchange (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (24) - : +- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (28) - +- * ColumnarToRow (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.customer (25) +* ColumnarToRow (36) ++- CometSort (35) + +- CometColumnarExchange (34) + +- RowToColumnar (33) + +- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * ColumnarToRow (26) + : +- CometFilter (25) + : +- CometHashAggregate (24) + : +- CometColumnarExchange (23) + : +- RowToColumnar (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (30) + +- * ColumnarToRow (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.customer (27) (unknown) Scan parquet spark_catalog.default.store_sales @@ -48,7 +52,7 @@ Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnu (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 37] +(4) ReusedExchange [Reuses operator id: 41] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -132,86 +136,96 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#14] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) Exchange +(22) RowToColumnar Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(23) HashAggregate [codegen id : 6] +(23) CometColumnarExchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(24) CometHashAggregate Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(24) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) +(25) CometFilter +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] +Condition : ((cnt#16 >= 15) AND (cnt#16 <= 20)) + +(26) ColumnarToRow [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] (unknown) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Output [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) +(28) CometFilter +Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Condition : isnotnull(c_customer_sk#17) -(27) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(29) ColumnarToRow [codegen id : 5] +Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] -(28) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(30) BroadcastExchange +Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#17] Join type: Inner Join condition: None -(30) Project [codegen id : 6] -Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(32) Project [codegen id : 6] +Output [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16, c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] + +(33) RowToColumnar +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] + +(34) CometColumnarExchange +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Arguments: rangepartitioning(c_last_name#20 ASC NULLS FIRST, c_first_name#19 ASC NULLS FIRST, c_salutation#18 ASC NULLS FIRST, c_preferred_cust_flag#21 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(31) Exchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(35) CometSort +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Arguments: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16], [c_last_name#20 ASC NULLS FIRST, c_first_name#19 ASC NULLS FIRST, c_salutation#18 ASC NULLS FIRST, c_preferred_cust_flag#21 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] -(32) Sort [codegen id : 7] -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST], true, 0 +(36) ColumnarToRow [codegen id : 7] +Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (41) ++- * ColumnarToRow (40) + +- CometProject (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.date_dim (37) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +Output [3]: [d_date_sk#7, d_year#22, d_dom#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] -Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(38) CometFilter +Input [3]: [d_date_sk#7, d_year#22, d_dom#23] +Condition : (((((d_dom#23 >= 1) AND (d_dom#23 <= 3)) OR ((d_dom#23 >= 25) AND (d_dom#23 <= 28))) AND d_year#22 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(35) CometProject -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +(39) CometProject +Input [3]: [d_date_sk#7, d_year#22, d_dom#23] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(40) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(37) BroadcastExchange +(41) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index b473e48921..9c94a51d26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -1,56 +1,60 @@ WholeStageCodegen (7) - Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] + ColumnarToRow InputAdapter - Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - InputAdapter - Exchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] + CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + RowToColumnar + WholeStageCodegen (6) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [cnt] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index e723b6c0e6..54878fd686 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +* ColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * Filter (25) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_demographics (33) (unknown) Scan parquet spark_catalog.default.customer @@ -67,7 +69,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) ReusedExchange [Reuses operator id: 47] +(6) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#9] (7) BroadcastHashJoin [codegen id : 2] @@ -100,7 +102,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(13) ReusedExchange [Reuses operator id: 47] +(13) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#13] (14) BroadcastHashJoin [codegen id : 4] @@ -133,7 +135,7 @@ ReadSchema: struct (19) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -(20) ReusedExchange [Reuses operator id: 47] +(20) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 6] @@ -227,50 +229,54 @@ Functions [10]: [partial_count(1), partial_avg(cd_dep_count#23), partial_max(cd_ Aggregate Attributes [13]: [count#26, sum#27, count#28, max#29, sum#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38] Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -(40) Exchange +(40) RowToColumnar Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) HashAggregate [codegen id : 10] +(41) CometColumnarExchange +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] +Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(42) CometHashAggregate Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] -Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] -Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] -(42) TakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] -Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] +(43) CometTakeOrderedAndProject +Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#19 ASC NULLS FIRST,cd_gender#21 ASC NULLS FIRST,cd_marital_status#22 ASC NULLS FIRST,cd_dep_count#23 ASC NULLS FIRST,cd_dep_employed_count#24 ASC NULLS FIRST,cd_dep_college_count#25 ASC NULLS FIRST], output=[ca_state#19,cd_gender#21,cd_marital_status#22,cd_dep_count#23,cnt1#52,avg(cd_dep_count)#53,max(cd_dep_count)#54,sum(cd_dep_count)#55,cd_dep_employed_count#24,cnt2#56,avg(cd_dep_employed_count)#57,max(cd_dep_employed_count)#58,sum(cd_dep_employed_count)#59,cd_dep_college_count#25,cnt3#60,avg(cd_dep_college_count)#61,max(cd_dep_college_count)#62,sum(cd_dep_college_count)#63]), 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] + +(44) ColumnarToRow [codegen id : 10] +Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#74, d_qoy#75] +Output [3]: [d_date_sk#9, d_year#64, d_qoy#65] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] -Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) +(46) CometFilter +Input [3]: [d_date_sk#9, d_year#64, d_qoy#65] +Condition : ((((isnotnull(d_year#64) AND isnotnull(d_qoy#65)) AND (d_year#64 = 2002)) AND (d_qoy#65 < 4)) AND isnotnull(d_date_sk#9)) -(45) CometProject -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +(47) CometProject +Input [3]: [d_date_sk#9, d_year#64, d_qoy#65] Arguments: [d_date_sk#9], [d_date_sk#9] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index dc724ca919..549322409a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -1,74 +1,76 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - InputAdapter - Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow +WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 29e62c0b79..d12bef6d51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -1,44 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.customer_demographics (31) +* ColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometHashAggregate (40) + +- CometColumnarExchange (39) + +- RowToColumnar (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : +- BroadcastExchange (9) + : : : +- * Project (8) + : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : :- * ColumnarToRow (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (6) + : : +- BroadcastExchange (22) + : : +- Union (21) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * ColumnarToRow (17) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (18) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.customer_address (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.customer_demographics (31) (unknown) Scan parquet spark_catalog.default.customer @@ -65,7 +67,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) ReusedExchange [Reuses operator id: 45] +(6) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#7] (7) BroadcastHashJoin [codegen id : 2] @@ -98,7 +100,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -(13) ReusedExchange [Reuses operator id: 45] +(13) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#11] (14) BroadcastHashJoin [codegen id : 4] @@ -121,7 +123,7 @@ ReadSchema: struct (17) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(18) ReusedExchange [Reuses operator id: 45] +(18) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#16] (19) BroadcastHashJoin [codegen id : 6] @@ -213,50 +215,54 @@ Functions [10]: [partial_count(1), partial_avg(cd_dep_count#23), partial_max(cd_ Aggregate Attributes [13]: [count#26, sum#27, count#28, max#29, sum#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38] Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -(38) Exchange +(38) RowToColumnar Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(39) HashAggregate [codegen id : 10] +(39) CometColumnarExchange +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] +Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(40) CometHashAggregate Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] -Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] -Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] -(40) TakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] -Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] +(41) CometTakeOrderedAndProject +Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#19 ASC NULLS FIRST,cd_gender#21 ASC NULLS FIRST,cd_marital_status#22 ASC NULLS FIRST,cd_dep_count#23 ASC NULLS FIRST,cd_dep_employed_count#24 ASC NULLS FIRST,cd_dep_college_count#25 ASC NULLS FIRST], output=[ca_state#19,cd_gender#21,cd_marital_status#22,cd_dep_count#23,cnt1#52,avg(cd_dep_count)#53,max(cd_dep_count)#54,sum(cd_dep_count)#55,cd_dep_employed_count#24,cnt2#56,avg(cd_dep_employed_count)#57,max(cd_dep_employed_count)#58,sum(cd_dep_employed_count)#59,cd_dep_college_count#25,cnt3#60,avg(cd_dep_college_count)#61,max(cd_dep_college_count)#62,sum(cd_dep_college_count)#63]), 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] + +(42) ColumnarToRow [codegen id : 10] +Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#74, d_qoy#75] +Output [3]: [d_date_sk#7, d_year#64, d_qoy#65] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] -Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 1999)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#7)) +(44) CometFilter +Input [3]: [d_date_sk#7, d_year#64, d_qoy#65] +Condition : ((((isnotnull(d_year#64) AND isnotnull(d_qoy#65)) AND (d_year#64 = 1999)) AND (d_qoy#65 < 4)) AND isnotnull(d_date_sk#7)) -(43) CometProject -Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] +(45) CometProject +Input [3]: [d_date_sk#7, d_year#64, d_qoy#65] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(45) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index e5cb940552..d5346d83b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -1,71 +1,73 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - InputAdapter - Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customsk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow +WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,customsk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + Union + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #5 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt index 2519f23a25..060abaefb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt @@ -1,45 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- Window (39) - +- * Sort (38) - +- Exchange (37) - +- * HashAggregate (36) - +- Exchange (35) - +- * HashAggregate (34) - +- Union (33) - :- * HashAggregate (22) - : +- Exchange (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.store (13) - :- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * HashAggregate (24) - : +- ReusedExchange (23) - +- * HashAggregate (32) - +- Exchange (31) - +- * HashAggregate (30) - +- * HashAggregate (29) - +- ReusedExchange (28) +TakeOrderedAndProject (43) ++- * Project (42) + +- Window (41) + +- * ColumnarToRow (40) + +- CometSort (39) + +- CometColumnarExchange (38) + +- CometHashAggregate (37) + +- CometColumnarExchange (36) + +- CometHashAggregate (35) + +- CometUnion (34) + :- CometHashAggregate (23) + : +- CometColumnarExchange (22) + : +- RowToColumnar (21) + : +- * HashAggregate (20) + : +- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.item (7) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.store (13) + :- CometHashAggregate (28) + : +- CometColumnarExchange (27) + : +- CometHashAggregate (26) + : +- CometHashAggregate (25) + : +- ReusedExchange (24) + +- CometHashAggregate (33) + +- CometColumnarExchange (32) + +- CometHashAggregate (31) + +- CometHashAggregate (30) + +- ReusedExchange (29) (unknown) Scan parquet spark_catalog.default.store_sales @@ -57,7 +59,7 @@ Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 46] +(4) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -137,142 +139,133 @@ Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(Unscale Aggregate Attributes [2]: [sum#13, sum#14] Results [4]: [i_category#10, i_class#9, sum#15, sum#16] -(21) Exchange +(21) RowToColumnar Input [4]: [i_category#10, i_class#9, sum#15, sum#16] -Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) HashAggregate [codegen id : 5] +(22) CometColumnarExchange +Input [4]: [i_category#10, i_class#9, sum#15, sum#16] +Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(23) CometHashAggregate Input [4]: [i_category#10, i_class#9, sum#15, sum#16] Keys [2]: [i_category#10, i_class#9] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#17, sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#17,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2)) as decimal(38,20)) AS gross_margin#19, i_category#10, i_class#9, 0 AS t_category#20, 0 AS t_class#21, 0 AS lochierarchy#22] -(23) ReusedExchange [Reuses operator id: 21] -Output [4]: [i_category#10, i_class#9, sum#23, sum#24] +(24) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#10, i_class#9, sum#17, sum#18] -(24) HashAggregate [codegen id : 10] -Input [4]: [i_category#10, i_class#9, sum#23, sum#24] +(25) CometHashAggregate +Input [4]: [i_category#10, i_class#9, sum#17, sum#18] Keys [2]: [i_category#10, i_class#9] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#25, sum(UnscaledValue(ss_ext_sales_price#3))#26] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#25,17,2) AS ss_net_profit#27, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#26,17,2) AS ss_ext_sales_price#28, i_category#10] -(25) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#27, ss_ext_sales_price#28, i_category#10] +(26) CometHashAggregate +Input [3]: [ss_net_profit#19, ss_ext_sales_price#20, i_category#10] Keys [1]: [i_category#10] -Functions [2]: [partial_sum(ss_net_profit#27), partial_sum(ss_ext_sales_price#28)] -Aggregate Attributes [4]: [sum#29, isEmpty#30, sum#31, isEmpty#32] -Results [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] +Functions [2]: [partial_sum(ss_net_profit#19), partial_sum(ss_ext_sales_price#20)] -(26) Exchange -Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(27) CometColumnarExchange +Input [5]: [i_category#10, sum#21, isEmpty#22, sum#23, isEmpty#24] +Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(27) HashAggregate [codegen id : 11] -Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] +(28) CometHashAggregate +Input [5]: [i_category#10, sum#21, isEmpty#22, sum#23, isEmpty#24] Keys [1]: [i_category#10] -Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)] -Aggregate Attributes [2]: [sum(ss_net_profit#27)#37, sum(ss_ext_sales_price#28)#38] -Results [6]: [cast((sum(ss_net_profit#27)#37 / sum(ss_ext_sales_price#28)#38) as decimal(38,20)) AS gross_margin#39, i_category#10, null AS i_class#40, 0 AS t_category#41, 1 AS t_class#42, 1 AS lochierarchy#43] +Functions [2]: [sum(ss_net_profit#19), sum(ss_ext_sales_price#20)] -(28) ReusedExchange [Reuses operator id: 21] -Output [4]: [i_category#10, i_class#9, sum#44, sum#45] +(29) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#10, i_class#9, sum#25, sum#26] -(29) HashAggregate [codegen id : 16] -Input [4]: [i_category#10, i_class#9, sum#44, sum#45] +(30) CometHashAggregate +Input [4]: [i_category#10, i_class#9, sum#25, sum#26] Keys [2]: [i_category#10, i_class#9] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#25, sum(UnscaledValue(ss_ext_sales_price#3))#26] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#25,17,2) AS ss_net_profit#27, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#26,17,2) AS ss_ext_sales_price#28] -(30) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#27, ss_ext_sales_price#28] +(31) CometHashAggregate +Input [2]: [ss_net_profit#19, ss_ext_sales_price#20] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#27), partial_sum(ss_ext_sales_price#28)] -Aggregate Attributes [4]: [sum#46, isEmpty#47, sum#48, isEmpty#49] -Results [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] +Functions [2]: [partial_sum(ss_net_profit#19), partial_sum(ss_ext_sales_price#20)] -(31) Exchange -Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] +(32) CometColumnarExchange +Input [4]: [sum#27, isEmpty#28, sum#29, isEmpty#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) HashAggregate [codegen id : 17] -Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] +(33) CometHashAggregate +Input [4]: [sum#27, isEmpty#28, sum#29, isEmpty#30] Keys: [] -Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)] -Aggregate Attributes [2]: [sum(ss_net_profit#27)#54, sum(ss_ext_sales_price#28)#55] -Results [6]: [cast((sum(ss_net_profit#27)#54 / sum(ss_ext_sales_price#28)#55) as decimal(38,20)) AS gross_margin#56, null AS i_category#57, null AS i_class#58, 1 AS t_category#59, 1 AS t_class#60, 2 AS lochierarchy#61] +Functions [2]: [sum(ss_net_profit#19), sum(ss_ext_sales_price#20)] -(33) Union +(34) CometUnion +Child 0 Input [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] +Child 1 Input [6]: [gross_margin#35, i_category#10, i_class#36, t_category#37, t_class#38, lochierarchy#39] +Child 2 Input [6]: [gross_margin#40, i_category#41, i_class#42, t_category#43, t_class#44, lochierarchy#45] -(34) HashAggregate [codegen id : 18] -Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +(35) CometHashAggregate +Input [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] +Keys [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] Functions: [] -Aggregate Attributes: [] -Results [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -(35) Exchange -Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -Arguments: hashpartitioning(gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(36) CometColumnarExchange +Input [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] +Arguments: hashpartitioning(gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(36) HashAggregate [codegen id : 19] -Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +(37) CometHashAggregate +Input [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] +Keys [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] Functions: [] -Aggregate Attributes: [] -Results [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, CASE WHEN (t_class#21 = 0) THEN i_category#10 END AS _w0#62] -(37) Exchange -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] -Arguments: hashpartitioning(lochierarchy#22, _w0#62, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(38) CometColumnarExchange +Input [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46] +Arguments: hashpartitioning(lochierarchy#34, _w0#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(39) CometSort +Input [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46] +Arguments: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46], [lochierarchy#34 ASC NULLS FIRST, _w0#46 ASC NULLS FIRST, gross_margin#31 ASC NULLS FIRST] -(38) Sort [codegen id : 20] -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] -Arguments: [lochierarchy#22 ASC NULLS FIRST, _w0#62 ASC NULLS FIRST, gross_margin#19 ASC NULLS FIRST], false, 0 +(40) ColumnarToRow [codegen id : 13] +Input [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46] -(39) Window -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] -Arguments: [rank(gross_margin#19) windowspecdefinition(lochierarchy#22, _w0#62, gross_margin#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#63], [lochierarchy#22, _w0#62], [gross_margin#19 ASC NULLS FIRST] +(41) Window +Input [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46] +Arguments: [rank(gross_margin#31) windowspecdefinition(lochierarchy#34, _w0#46, gross_margin#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#47], [lochierarchy#34, _w0#46], [gross_margin#31 ASC NULLS FIRST] -(40) Project [codegen id : 21] -Output [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] -Input [6]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62, rank_within_parent#63] +(42) Project [codegen id : 14] +Output [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, rank_within_parent#47] +Input [6]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46, rank_within_parent#47] -(41) TakeOrderedAndProject -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] -Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] +(43) TakeOrderedAndProject +Input [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, rank_within_parent#47] +Arguments: 100, [lochierarchy#34 DESC NULLS LAST, CASE WHEN (lochierarchy#34 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, rank_within_parent#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (46) -+- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.date_dim (42) +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#64] +Output [2]: [d_date_sk#7, d_year#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter -Input [2]: [d_date_sk#7, d_year#64] -Condition : ((isnotnull(d_year#64) AND (d_year#64 = 2001)) AND isnotnull(d_date_sk#7)) +(45) CometFilter +Input [2]: [d_date_sk#7, d_year#48] +Condition : ((isnotnull(d_year#48) AND (d_year#48 = 2001)) AND isnotnull(d_date_sk#7)) -(44) CometProject -Input [2]: [d_date_sk#7, d_year#64] +(46) CometProject +Input [2]: [d_date_sk#7, d_year#48] Arguments: [d_date_sk#7], [d_date_sk#7] -(45) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(46) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index f265d20995..29ae8ca601 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -1,76 +1,64 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (21) + WholeStageCodegen (14) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (20) - Sort [lochierarchy,_w0,gross_margin] + WholeStageCodegen (13) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (19) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] [_w0] - InputAdapter - Exchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,t_category,t_class,lochierarchy,sum,sum] - InputAdapter - Exchange [i_category,i_class] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometSort [lochierarchy,_w0,gross_margin] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometUnion + CometHashAggregate [i_category,i_class,sum,sum] + CometColumnarExchange [i_category,i_class] #3 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (11) - HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [i_category] #7 - WholeStageCodegen (10) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange #8 - WholeStageCodegen (16) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometHashAggregate [i_category,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [i_category] #7 + CometHashAggregate [i_category,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,sum,sum] + ReusedExchange [i_category,i_class,sum,sum] #3 + CometHashAggregate [sum,isEmpty,sum,isEmpty] + CometColumnarExchange #8 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,sum,sum] + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index 2be3c9c660..aa410a6b55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * Filter (25) - : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) - +- BroadcastExchange (42) - +- * Project (41) - +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) +TakeOrderedAndProject (47) ++- * Project (46) + +- * BroadcastHashJoin Inner BuildRight (45) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * ColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- CometHashAggregate (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- BroadcastExchange (36) + : +- * Project (35) + : +- Window (34) + : +- * ColumnarToRow (33) + : +- CometSort (32) + : +- ReusedExchange (31) + +- BroadcastExchange (44) + +- * Project (43) + +- Window (42) + +- * ColumnarToRow (41) + +- CometSort (40) + +- ReusedExchange (39) (unknown) Scan parquet spark_catalog.default.item @@ -89,7 +91,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(10) ReusedExchange [Reuses operator id: 51] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -137,125 +139,124 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [1]: [sum#15] Results [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -(20) Exchange +(20) RowToColumnar Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) HashAggregate [codegen id : 5] +(21) CometColumnarExchange +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) CometHashAggregate Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#17] -Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS _w0#19] -(22) Exchange -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(23) CometColumnarExchange +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(23) Sort [codegen id : 6] -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 +(24) CometSort +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(24) Window -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(25) ColumnarToRow [codegen id : 5] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] + +(26) Window +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] +(27) Filter [codegen id : 6] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(26) Window -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] - -(27) Filter [codegen id : 22] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(28) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(29) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(30) HashAggregate [codegen id : 12] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] -Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] - -(31) Exchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(32) Sort [codegen id : 13] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 - -(33) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(34) Project [codegen id : 14] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] - -(35) BroadcastExchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] - -(36) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] +(28) Window +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] + +(29) Filter [codegen id : 19] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) + +(30) Project [codegen id : 19] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] + +(31) ReusedExchange [Reuses operator id: 23] +Output [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] + +(32) CometSort +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, s_store_name#23 ASC NULLS FIRST, s_company_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] + +(33) ColumnarToRow [codegen id : 11] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] + +(34) Window +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#27], [i_category#21, i_brand#22, s_store_name#23, s_company_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] + +(35) Project [codegen id : 12] +Output [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#17 AS sum_sales#28, rn#27] +Input [8]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17, rn#27] + +(36) BroadcastExchange +Input [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#28, rn#27] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 19] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] +Right keys [5]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, (rn#27 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +(38) Project [codegen id : 19] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#28, rn#27] + +(39) ReusedExchange [Reuses operator id: 23] +Output [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] -(38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +(40) CometSort +Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] +Arguments: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17], [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, s_store_name#31 ASC NULLS FIRST, s_company_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(39) Sort [codegen id : 20] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 +(41) ColumnarToRow [codegen id : 17] +Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] -(40) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +(42) Window +Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#29, i_brand#30, s_store_name#31, s_company_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(41) Project [codegen id : 21] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] +(43) Project [codegen id : 18] +Output [6]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#17 AS sum_sales#36, rn#35] +Input [8]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] -(42) BroadcastExchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] +(44) BroadcastExchange +Input [6]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#36, rn#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] +(45) BroadcastHashJoin [codegen id : 19] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] +Right keys [5]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] -Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +(46) Project [codegen id : 19] +Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#36 AS nsum#38] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#36, rn#35] -(45) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +(47) TakeOrderedAndProject +Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim @@ -265,15 +266,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(49) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(51) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index a548953052..d3aeefb861 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (19) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,18 +8,18 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (6) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + RowToColumnar WholeStageCodegen (4) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] @@ -57,25 +57,23 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (11) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #7 + WholeStageCodegen (18) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + WholeStageCodegen (17) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index f8419179d4..7d3280c095 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -1,78 +1,86 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- Union (70) - :- * Project (23) - : +- * Filter (22) - : +- Window (21) - : +- * Sort (20) - : +- Window (19) - : +- * Sort (18) - : +- Exchange (17) - : +- * HashAggregate (16) - : +- Exchange (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.web_returns (5) - : +- ReusedExchange (11) - :- * Project (46) - : +- * Filter (45) - : +- Window (44) - : +- * Sort (43) - : +- Window (42) - : +- * Sort (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometScan parquet spark_catalog.default.catalog_returns (28) - : +- ReusedExchange (34) - +- * Project (69) - +- * Filter (68) - +- Window (67) - +- * Sort (66) - +- Window (65) - +- * Sort (64) - +- Exchange (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * ColumnarToRow (56) - : +- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometBroadcastExchange (50) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan parquet spark_catalog.default.store_sales (47) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.store_returns (51) - +- ReusedExchange (57) +* ColumnarToRow (82) ++- CometTakeOrderedAndProject (81) + +- CometHashAggregate (80) + +- CometColumnarExchange (79) + +- RowToColumnar (78) + +- * HashAggregate (77) + +- Union (76) + :- * Project (25) + : +- * Filter (24) + : +- Window (23) + : +- * Sort (22) + : +- Window (21) + : +- * ColumnarToRow (20) + : +- CometSort (19) + : +- CometColumnarExchange (18) + : +- CometHashAggregate (17) + : +- CometColumnarExchange (16) + : +- RowToColumnar (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- ReusedExchange (11) + :- * Project (50) + : +- * Filter (49) + : +- Window (48) + : +- * Sort (47) + : +- Window (46) + : +- * ColumnarToRow (45) + : +- CometSort (44) + : +- CometColumnarExchange (43) + : +- CometHashAggregate (42) + : +- CometColumnarExchange (41) + : +- RowToColumnar (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * ColumnarToRow (35) + : : +- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometBroadcastExchange (29) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (26) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (36) + +- * Project (75) + +- * Filter (74) + +- Window (73) + +- * Sort (72) + +- Window (71) + +- * ColumnarToRow (70) + +- CometSort (69) + +- CometColumnarExchange (68) + +- CometHashAggregate (67) + +- CometColumnarExchange (66) + +- RowToColumnar (65) + +- * HashAggregate (64) + +- * Project (63) + +- * BroadcastHashJoin Inner BuildRight (62) + :- * ColumnarToRow (60) + : +- CometProject (59) + : +- CometBroadcastHashJoin (58) + : :- CometBroadcastExchange (54) + : : +- CometProject (53) + : : +- CometFilter (52) + : : +- CometScan parquet spark_catalog.default.store_sales (51) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.store_returns (55) + +- ReusedExchange (61) (unknown) Scan parquet spark_catalog.default.web_sales @@ -122,7 +130,7 @@ Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_re (10) ColumnarToRow [codegen id : 2] Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(11) ReusedExchange [Reuses operator id: 79] +(11) ReusedExchange [Reuses operator id: 87] Output [1]: [d_date_sk#13] (12) BroadcastHashJoin [codegen id : 2] @@ -142,315 +150,331 @@ Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coa Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(15) Exchange +(15) RowToColumnar Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(16) HashAggregate [codegen id : 3] +(16) CometColumnarExchange +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(17) CometHashAggregate Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] -(17) Exchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +(18) CometColumnarExchange +Input [3]: [item#26, return_ratio#27, currency_ratio#28] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(18) Sort [codegen id : 4] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 +(19) CometSort +Input [3]: [item#26, return_ratio#27, currency_ratio#28] +Arguments: [item#26, return_ratio#27, currency_ratio#28], [return_ratio#27 ASC NULLS FIRST] -(19) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] - -(20) Sort [codegen id : 5] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 +(20) ColumnarToRow [codegen id : 3] +Input [3]: [item#26, return_ratio#27, currency_ratio#28] (21) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] +Input [3]: [item#26, return_ratio#27, currency_ratio#28] +Arguments: [rank(return_ratio#27) windowspecdefinition(return_ratio#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#29], [return_ratio#27 ASC NULLS FIRST] + +(22) Sort [codegen id : 4] +Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] +Arguments: [currency_ratio#28 ASC NULLS FIRST], false, 0 -(22) Filter [codegen id : 6] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) +(23) Window +Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] +Arguments: [rank(currency_ratio#28) windowspecdefinition(currency_ratio#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#30], [currency_ratio#28 ASC NULLS FIRST] -(23) Project [codegen id : 6] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +(24) Filter [codegen id : 5] +Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] +Condition : ((return_rank#29 <= 10) OR (currency_rank#30 <= 10)) + +(25) Project [codegen id : 5] +Output [5]: [web AS channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Output [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(25) CometFilter -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) +(27) CometFilter +Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] +Condition : (((((((isnotnull(cs_net_profit#36) AND isnotnull(cs_net_paid#35)) AND isnotnull(cs_quantity#34)) AND (cs_net_profit#36 > 1.00)) AND (cs_net_paid#35 > 0.00)) AND (cs_quantity#34 > 0)) AND isnotnull(cs_order_number#33)) AND isnotnull(cs_item_sk#32)) -(26) CometProject -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(28) CometProject +Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] +Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37], [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] -(27) CometBroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(29) CometBroadcastExchange +Input [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Output [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(29) CometFilter -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) +(31) CometFilter +Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] +Condition : (((isnotnull(cr_return_amount#42) AND (cr_return_amount#42 > 10000.00)) AND isnotnull(cr_order_number#40)) AND isnotnull(cr_item_sk#39)) -(30) CometProject -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +(32) CometProject +Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] +Arguments: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42], [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] -(31) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner +(33) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +Right output [4]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] +Arguments: [cs_order_number#33, cs_item_sk#32], [cr_order_number#40, cr_item_sk#39], Inner -(32) CometProject -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] +(34) CometProject +Input [9]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] +Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] -(33) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] +(35) ColumnarToRow [codegen id : 7] +Input [6]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] -(34) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#48] +(36) ReusedExchange [Reuses operator id: 87] +Output [1]: [d_date_sk#44] -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#48] +(37) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_sold_date_sk#37] +Right keys [1]: [d_date_sk#44] Join type: Inner Join condition: None -(36) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] +(38) Project [codegen id : 7] +Output [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] +Input [7]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42, d_date_sk#44] + +(39) HashAggregate [codegen id : 7] +Input [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] +Keys [1]: [cs_item_sk#32] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#41, 0)), partial_sum(coalesce(cs_quantity#34, 0)), partial_sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] +Results [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] -(37) HashAggregate [codegen id : 8] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] -Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +(40) RowToColumnar +Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] -(38) Exchange -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(41) CometColumnarExchange +Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] +Arguments: hashpartitioning(cs_item_sk#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(39) HashAggregate [codegen id : 9] -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] -Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] +(42) CometHashAggregate +Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] +Keys [1]: [cs_item_sk#32] +Functions [4]: [sum(coalesce(cr_return_quantity#41, 0)), sum(coalesce(cs_quantity#34, 0)), sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] -(40) Exchange -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(43) CometColumnarExchange +Input [3]: [item#57, return_ratio#58, currency_ratio#59] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) Sort [codegen id : 10] -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 +(44) CometSort +Input [3]: [item#57, return_ratio#58, currency_ratio#59] +Arguments: [item#57, return_ratio#58, currency_ratio#59], [return_ratio#58 ASC NULLS FIRST] -(42) Window -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] +(45) ColumnarToRow [codegen id : 8] +Input [3]: [item#57, return_ratio#58, currency_ratio#59] -(43) Sort [codegen id : 11] -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 +(46) Window +Input [3]: [item#57, return_ratio#58, currency_ratio#59] +Arguments: [rank(return_ratio#58) windowspecdefinition(return_ratio#58 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#60], [return_ratio#58 ASC NULLS FIRST] -(44) Window -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] +(47) Sort [codegen id : 9] +Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] +Arguments: [currency_ratio#59 ASC NULLS FIRST], false, 0 -(45) Filter [codegen id : 12] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] -Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) +(48) Window +Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] +Arguments: [rank(currency_ratio#59) windowspecdefinition(currency_ratio#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#61], [currency_ratio#59 ASC NULLS FIRST] -(46) Project [codegen id : 12] -Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +(49) Filter [codegen id : 10] +Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] +Condition : ((return_rank#60 <= 10) OR (currency_rank#61 <= 10)) + +(50) Project [codegen id : 10] +Output [5]: [catalog AS channel#62, item#57, return_ratio#58, return_rank#60, currency_rank#61] +Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] (unknown) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Output [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#68), dynamicpruningexpression(ss_sold_date_sk#68 IN dynamicpruning#69)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(48) CometFilter -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) +(52) CometFilter +Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] +Condition : (((((((isnotnull(ss_net_profit#67) AND isnotnull(ss_net_paid#66)) AND isnotnull(ss_quantity#65)) AND (ss_net_profit#67 > 1.00)) AND (ss_net_paid#66 > 0.00)) AND (ss_quantity#65 > 0)) AND isnotnull(ss_ticket_number#64)) AND isnotnull(ss_item_sk#63)) -(49) CometProject -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(53) CometProject +Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] +Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68], [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] -(50) CometBroadcastExchange -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(54) CometBroadcastExchange +Input [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] (unknown) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Output [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(52) CometFilter -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) +(56) CometFilter +Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] +Condition : (((isnotnull(sr_return_amt#73) AND (sr_return_amt#73 > 10000.00)) AND isnotnull(sr_ticket_number#71)) AND isnotnull(sr_item_sk#70)) -(53) CometProject -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +(57) CometProject +Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] +Arguments: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73], [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] -(54) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner +(58) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +Right output [4]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] +Arguments: [ss_ticket_number#64, ss_item_sk#63], [sr_ticket_number#71, sr_item_sk#70], Inner -(55) CometProject -Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] +(59) CometProject +Input [9]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] +Arguments: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73], [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] -(56) ColumnarToRow [codegen id : 14] -Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] +(60) ColumnarToRow [codegen id : 12] +Input [6]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] -(57) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#83] +(61) ReusedExchange [Reuses operator id: 87] +Output [1]: [d_date_sk#75] -(58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#76] -Right keys [1]: [d_date_sk#83] +(62) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [ss_sold_date_sk#68] +Right keys [1]: [d_date_sk#75] Join type: Inner Join condition: None -(59) Project [codegen id : 14] -Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] - -(60) HashAggregate [codegen id : 14] -Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Keys [1]: [ss_item_sk#71] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] -Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(61) Exchange -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(62) HashAggregate [codegen id : 15] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Keys [1]: [ss_item_sk#71] -Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] -Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] - -(63) Exchange -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] - -(64) Sort [codegen id : 16] -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 - -(65) Window -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] - -(66) Sort [codegen id : 17] -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 - -(67) Window -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] - -(68) Filter [codegen id : 18] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) - -(69) Project [codegen id : 18] -Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] - -(70) Union - -(71) HashAggregate [codegen id : 19] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(63) Project [codegen id : 12] +Output [5]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] +Input [7]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73, d_date_sk#75] + +(64) HashAggregate [codegen id : 12] +Input [5]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] +Keys [1]: [ss_item_sk#63] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#72, 0)), partial_sum(coalesce(ss_quantity#65, 0)), partial_sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#76, sum#77, sum#78, isEmpty#79, sum#80, isEmpty#81] +Results [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(65) RowToColumnar +Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(66) CometColumnarExchange +Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Arguments: hashpartitioning(ss_item_sk#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(67) CometHashAggregate +Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [1]: [ss_item_sk#63] +Functions [4]: [sum(coalesce(sr_return_quantity#72, 0)), sum(coalesce(ss_quantity#65, 0)), sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] + +(68) CometColumnarExchange +Input [3]: [item#88, return_ratio#89, currency_ratio#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(69) CometSort +Input [3]: [item#88, return_ratio#89, currency_ratio#90] +Arguments: [item#88, return_ratio#89, currency_ratio#90], [return_ratio#89 ASC NULLS FIRST] + +(70) ColumnarToRow [codegen id : 13] +Input [3]: [item#88, return_ratio#89, currency_ratio#90] + +(71) Window +Input [3]: [item#88, return_ratio#89, currency_ratio#90] +Arguments: [rank(return_ratio#89) windowspecdefinition(return_ratio#89 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#91], [return_ratio#89 ASC NULLS FIRST] + +(72) Sort [codegen id : 14] +Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] +Arguments: [currency_ratio#90 ASC NULLS FIRST], false, 0 + +(73) Window +Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] +Arguments: [rank(currency_ratio#90) windowspecdefinition(currency_ratio#90 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#92], [currency_ratio#90 ASC NULLS FIRST] + +(74) Filter [codegen id : 15] +Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] +Condition : ((return_rank#91 <= 10) OR (currency_rank#92 <= 10)) + +(75) Project [codegen id : 15] +Output [5]: [store AS channel#93, item#88, return_ratio#89, return_rank#91, currency_rank#92] +Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] + +(76) Union + +(77) HashAggregate [codegen id : 16] +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(72) Exchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(78) RowToColumnar +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(73) HashAggregate [codegen id : 20] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(79) CometColumnarExchange +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Arguments: hashpartitioning(channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(80) CometHashAggregate +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(74) TakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(81) CometTakeOrderedAndProject +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,return_rank#29 ASC NULLS FIRST,currency_rank#30 ASC NULLS FIRST,item#26 ASC NULLS FIRST], output=[channel#31,item#26,return_ratio#27,return_rank#29,currency_rank#30]), 100, [channel#31 ASC NULLS FIRST, return_rank#29 ASC NULLS FIRST, currency_rank#30 ASC NULLS FIRST, item#26 ASC NULLS FIRST], [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] + +(82) ColumnarToRow [codegen id : 17] +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan parquet spark_catalog.default.date_dim (75) +BroadcastExchange (87) ++- * ColumnarToRow (86) + +- CometProject (85) + +- CometFilter (84) + +- CometScan parquet spark_catalog.default.date_dim (83) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#106, d_moy#107] +Output [3]: [d_date_sk#13, d_year#94, d_moy#95] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] -Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) +(84) CometFilter +Input [3]: [d_date_sk#13, d_year#94, d_moy#95] +Condition : ((((isnotnull(d_year#94) AND isnotnull(d_moy#95)) AND (d_year#94 = 2001)) AND (d_moy#95 = 12)) AND isnotnull(d_date_sk#13)) -(77) CometProject -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +(85) CometProject +Input [3]: [d_date_sk#13, d_year#94, d_moy#95] Arguments: [d_date_sk#13], [d_date_sk#13] -(78) ColumnarToRow [codegen id : 1] +(86) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(79) BroadcastExchange +(87) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#68 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index 43ebf34cc0..6ea9407566 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -1,121 +1,123 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (20) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (19) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - Sort [return_ratio] - InputAdapter - Exchange #2 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (12) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (11) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (10) - Sort [return_ratio] - InputAdapter - Exchange #6 - WholeStageCodegen (9) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #7 - WholeStageCodegen (8) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #8 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (18) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (17) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (16) - Sort [return_ratio] - InputAdapter - Exchange #9 - WholeStageCodegen (15) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #11 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 +WholeStageCodegen (17) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (5) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (4) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometSort [return_ratio] + CometColumnarExchange #2 + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [ws_item_sk] #3 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (10) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometSort [return_ratio] + CometColumnarExchange #6 + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [cs_item_sk] #7 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #8 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (15) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (14) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (13) + ColumnarToRow + InputAdapter + CometSort [return_ratio] + CometColumnarExchange #9 + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [ss_item_sk] #10 + RowToColumnar + WholeStageCodegen (12) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #11 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index 0dedd04721..60f983923a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -1,71 +1,81 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Filter (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- Window (56) - : +- * Sort (55) - : +- Exchange (54) - : +- * Project (53) - : +- * Filter (52) - : +- * SortMergeJoin FullOuter (51) - : :- * Sort (25) - : : +- Exchange (24) - : : +- * HashAggregate (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (13) - : : : +- Window (12) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (18) - : : +- * Project (17) - : : +- Window (16) - : : +- * Sort (15) - : : +- ReusedExchange (14) - : +- * Sort (50) - : +- Exchange (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (38) - : : +- Window (37) - : : +- * Sort (36) - : : +- Exchange (35) - : : +- * HashAggregate (34) - : : +- Exchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- CometFilter (27) - : : : +- CometScan parquet spark_catalog.default.store_sales (26) - : : +- ReusedExchange (29) - : +- BroadcastExchange (43) - : +- * Project (42) - : +- Window (41) - : +- * Sort (40) - : +- ReusedExchange (39) - +- BroadcastExchange (61) - +- * Project (60) - +- Window (59) - +- * Sort (58) - +- ReusedExchange (57) +TakeOrderedAndProject (77) ++- * Filter (76) + +- * HashAggregate (75) + +- * HashAggregate (74) + +- * Project (73) + +- * BroadcastHashJoin Inner BuildRight (72) + :- Window (65) + : +- * ColumnarToRow (64) + : +- CometSort (63) + : +- CometColumnarExchange (62) + : +- CometProject (61) + : +- CometFilter (60) + : +- CometSortMergeJoin (59) + : :- CometSort (29) + : : +- CometColumnarExchange (28) + : : +- CometHashAggregate (27) + : : +- CometColumnarExchange (26) + : : +- RowToColumnar (25) + : : +- * HashAggregate (24) + : : +- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Project (15) + : : : +- Window (14) + : : : +- * ColumnarToRow (13) + : : : +- CometSort (12) + : : : +- CometColumnarExchange (11) + : : : +- CometHashAggregate (10) + : : : +- CometColumnarExchange (9) + : : : +- RowToColumnar (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (21) + : : +- * Project (20) + : : +- Window (19) + : : +- * ColumnarToRow (18) + : : +- CometSort (17) + : : +- ReusedExchange (16) + : +- CometSort (58) + : +- CometColumnarExchange (57) + : +- CometHashAggregate (56) + : +- CometColumnarExchange (55) + : +- RowToColumnar (54) + : +- * HashAggregate (53) + : +- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (44) + : : +- Window (43) + : : +- * ColumnarToRow (42) + : : +- CometSort (41) + : : +- CometColumnarExchange (40) + : : +- CometHashAggregate (39) + : : +- CometColumnarExchange (38) + : : +- RowToColumnar (37) + : : +- * HashAggregate (36) + : : +- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * ColumnarToRow (32) + : : : +- CometFilter (31) + : : : +- CometScan parquet spark_catalog.default.store_sales (30) + : : +- ReusedExchange (33) + : +- BroadcastExchange (50) + : +- * Project (49) + : +- Window (48) + : +- * ColumnarToRow (47) + : +- CometSort (46) + : +- ReusedExchange (45) + +- BroadcastExchange (71) + +- * Project (70) + +- Window (69) + +- * ColumnarToRow (68) + +- CometSort (67) + +- ReusedExchange (66) (unknown) Scan parquet spark_catalog.default.web_sales @@ -83,7 +93,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 2] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 72] +(4) ReusedExchange [Reuses operator id: 82] Output [2]: [d_date_sk#5, d_date#6] (5) BroadcastHashJoin [codegen id : 2] @@ -103,314 +113,335 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] Aggregate Attributes [1]: [sum#7] Results [3]: [ws_item_sk#1, d_date#6, sum#8] -(8) Exchange +(8) RowToColumnar Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) HashAggregate [codegen id : 3] +(9) CometColumnarExchange +Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] + +(10) CometHashAggregate Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] -(10) Exchange -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(11) CometColumnarExchange +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(12) CometSort +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(11) Sort [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 +(13) ColumnarToRow [codegen id : 3] +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -(12) Window -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(14) Window +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(13) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] -Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] +(15) Project [codegen id : 8] +Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] +Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] -(14) ReusedExchange [Reuses operator id: 10] -Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +(16) ReusedExchange [Reuses operator id: 11] +Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(15) Sort [codegen id : 8] -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST], false, 0 +(17) CometSort +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] -(16) Window -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] +(18) ColumnarToRow [codegen id : 6] +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(17) Project [codegen id : 9] -Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] -Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] +(19) Window +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] -(18) BroadcastExchange -Input [3]: [item_sk#16, sumws#17, rk#15] +(20) Project [codegen id : 7] +Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] +Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] + +(21) BroadcastExchange +Input [3]: [item_sk#15, sumws#16, rk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#16] +(22) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [item_sk#9] +Right keys [1]: [item_sk#15] Join type: Inner -Join condition: (rk#12 >= rk#15) - -(20) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] - -(21) HashAggregate [codegen id : 10] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#17)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(22) Exchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(23) HashAggregate [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [sum(sumws#17)] -Aggregate Attributes [1]: [sum(sumws#17)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] - -(24) Exchange -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(25) Sort [codegen id : 12] -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 +Join condition: (rk#11 >= rk#14) + +(23) Project [codegen id : 8] +Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] + +(24) HashAggregate [codegen id : 8] +Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [partial_sum(sumws#16)] +Aggregate Attributes [2]: [sum#17, isEmpty#18] +Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(25) RowToColumnar +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(26) CometColumnarExchange +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(27) CometHashAggregate +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [sum(sumws#16)] + +(28) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#21] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(29) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#21] +Arguments: [item_sk#9, d_date#6, cume_sales#21], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Output [3]: [ss_item_sk#22, ss_sales_price#23, ss_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#25)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(27) CometFilter -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) +(31) CometFilter +Input [3]: [ss_item_sk#22, ss_sales_price#23, ss_sold_date_sk#24] +Condition : isnotnull(ss_item_sk#22) -(28) ColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +(32) ColumnarToRow [codegen id : 10] +Input [3]: [ss_item_sk#22, ss_sales_price#23, ss_sold_date_sk#24] -(29) ReusedExchange [Reuses operator id: 72] -Output [2]: [d_date_sk#28, d_date#29] +(33) ReusedExchange [Reuses operator id: 82] +Output [2]: [d_date_sk#26, d_date#27] -(30) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#28] +(34) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#24] +Right keys [1]: [d_date_sk#26] Join type: Inner Join condition: None -(31) Project [codegen id : 14] -Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] -Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] - -(32) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#30] -Results [3]: [ss_item_sk#24, d_date#29, sum#31] - -(33) Exchange -Input [3]: [ss_item_sk#24, d_date#29, sum#31] -Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(34) HashAggregate [codegen id : 15] -Input [3]: [ss_item_sk#24, d_date#29, sum#31] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#32] -Results [4]: [ss_item_sk#24 AS item_sk#33, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#32,17,2) AS sumss#34, ss_item_sk#24] - -(35) Exchange -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(36) Sort [codegen id : 16] -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 - -(37) Window -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] - -(38) Project [codegen id : 22] -Output [4]: [item_sk#33, d_date#29, sumss#34, rk#35] -Input [5]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24, rk#35] - -(39) ReusedExchange [Reuses operator id: 35] -Output [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] - -(40) Sort [codegen id : 20] -Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -Arguments: [ss_item_sk#37 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST], false, 0 - -(41) Window -Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -Arguments: [row_number() windowspecdefinition(ss_item_sk#37, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [ss_item_sk#37], [d_date#36 ASC NULLS FIRST] - -(42) Project [codegen id : 21] -Output [3]: [item_sk#33 AS item_sk#39, sumss#34 AS sumss#40, rk#38] -Input [5]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37, rk#38] - -(43) BroadcastExchange -Input [3]: [item_sk#39, sumss#40, rk#38] +(35) Project [codegen id : 10] +Output [3]: [ss_item_sk#22, ss_sales_price#23, d_date#27] +Input [5]: [ss_item_sk#22, ss_sales_price#23, ss_sold_date_sk#24, d_date_sk#26, d_date#27] + +(36) HashAggregate [codegen id : 10] +Input [3]: [ss_item_sk#22, ss_sales_price#23, d_date#27] +Keys [2]: [ss_item_sk#22, d_date#27] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#23))] +Aggregate Attributes [1]: [sum#28] +Results [3]: [ss_item_sk#22, d_date#27, sum#29] + +(37) RowToColumnar +Input [3]: [ss_item_sk#22, d_date#27, sum#29] + +(38) CometColumnarExchange +Input [3]: [ss_item_sk#22, d_date#27, sum#29] +Arguments: hashpartitioning(ss_item_sk#22, d_date#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(39) CometHashAggregate +Input [3]: [ss_item_sk#22, d_date#27, sum#29] +Keys [2]: [ss_item_sk#22, d_date#27] +Functions [1]: [sum(UnscaledValue(ss_sales_price#23))] + +(40) CometColumnarExchange +Input [4]: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22] +Arguments: hashpartitioning(ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(41) CometSort +Input [4]: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22] +Arguments: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22], [ss_item_sk#22 ASC NULLS FIRST, d_date#27 ASC NULLS FIRST] + +(42) ColumnarToRow [codegen id : 11] +Input [4]: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22] + +(43) Window +Input [4]: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22] +Arguments: [row_number() windowspecdefinition(ss_item_sk#22, d_date#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#22], [d_date#27 ASC NULLS FIRST] + +(44) Project [codegen id : 16] +Output [4]: [item_sk#30, d_date#27, sumss#31, rk#32] +Input [5]: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22, rk#32] + +(45) ReusedExchange [Reuses operator id: 40] +Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] + +(46) CometSort +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] + +(47) ColumnarToRow [codegen id : 14] +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] + +(48) Window +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] + +(49) Project [codegen id : 15] +Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] +Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] + +(50) BroadcastExchange +Input [3]: [item_sk#36, sumss#37, rk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(44) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [item_sk#33] -Right keys [1]: [item_sk#39] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [item_sk#30] +Right keys [1]: [item_sk#36] Join type: Inner -Join condition: (rk#35 >= rk#38) - -(45) Project [codegen id : 22] -Output [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] -Input [7]: [item_sk#33, d_date#29, sumss#34, rk#35, item_sk#39, sumss#40, rk#38] - -(46) HashAggregate [codegen id : 22] -Input [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] -Keys [3]: [item_sk#33, d_date#29, sumss#34] -Functions [1]: [partial_sum(sumss#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] - -(47) Exchange -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -Arguments: hashpartitioning(item_sk#33, d_date#29, sumss#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(48) HashAggregate [codegen id : 23] -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -Keys [3]: [item_sk#33, d_date#29, sumss#34] -Functions [1]: [sum(sumss#40)] -Aggregate Attributes [1]: [sum(sumss#40)#45] -Results [3]: [item_sk#33, d_date#29, sum(sumss#40)#45 AS cume_sales#46] - -(49) Exchange -Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: hashpartitioning(item_sk#33, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(50) Sort [codegen id : 24] -Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#33 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 - -(51) SortMergeJoin [codegen id : 25] -Left keys [2]: [item_sk#10, d_date#6] -Right keys [2]: [item_sk#33, d_date#29] -Join type: FullOuter -Join condition: None +Join condition: (rk#32 >= rk#35) + +(52) Project [codegen id : 16] +Output [4]: [item_sk#30, d_date#27, sumss#31, sumss#37] +Input [7]: [item_sk#30, d_date#27, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] + +(53) HashAggregate [codegen id : 16] +Input [4]: [item_sk#30, d_date#27, sumss#31, sumss#37] +Keys [3]: [item_sk#30, d_date#27, sumss#31] +Functions [1]: [partial_sum(sumss#37)] +Aggregate Attributes [2]: [sum#38, isEmpty#39] +Results [5]: [item_sk#30, d_date#27, sumss#31, sum#40, isEmpty#41] + +(54) RowToColumnar +Input [5]: [item_sk#30, d_date#27, sumss#31, sum#40, isEmpty#41] + +(55) CometColumnarExchange +Input [5]: [item_sk#30, d_date#27, sumss#31, sum#40, isEmpty#41] +Arguments: hashpartitioning(item_sk#30, d_date#27, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(56) CometHashAggregate +Input [5]: [item_sk#30, d_date#27, sumss#31, sum#40, isEmpty#41] +Keys [3]: [item_sk#30, d_date#27, sumss#31] +Functions [1]: [sum(sumss#37)] + +(57) CometColumnarExchange +Input [3]: [item_sk#30, d_date#27, cume_sales#42] +Arguments: hashpartitioning(item_sk#30, d_date#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(58) CometSort +Input [3]: [item_sk#30, d_date#27, cume_sales#42] +Arguments: [item_sk#30, d_date#27, cume_sales#42], [item_sk#30 ASC NULLS FIRST, d_date#27 ASC NULLS FIRST] + +(59) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#21] +Right output [3]: [item_sk#30, d_date#27, cume_sales#42] +Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#27], FullOuter + +(60) CometFilter +Input [6]: [item_sk#9, d_date#6, cume_sales#21, item_sk#30, d_date#27, cume_sales#42] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) + +(61) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#21, item_sk#30, d_date#27, cume_sales#42] +Arguments: [item_sk#43, d_date#44, web_sales#45, store_sales#46], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#43, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#27 END AS d_date#44, cume_sales#21 AS web_sales#45, cume_sales#42 AS store_sales#46] -(52) Filter [codegen id : 25] -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END) +(62) CometColumnarExchange +Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] +Arguments: hashpartitioning(item_sk#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(53) Project [codegen id : 25] -Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END AS item_sk#47, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#46 AS store_sales#50] -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] +(63) CometSort +Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] +Arguments: [item_sk#43, d_date#44, web_sales#45, store_sales#46], [item_sk#43 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST] -(54) Exchange -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: hashpartitioning(item_sk#47, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(64) ColumnarToRow [codegen id : 17] +Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] -(55) Sort [codegen id : 26] -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], false, 0 +(65) Window +Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] +Arguments: [row_number() windowspecdefinition(item_sk#43, d_date#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#47], [item_sk#43], [d_date#44 ASC NULLS FIRST] -(56) Window -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#47], [d_date#48 ASC NULLS FIRST] +(66) ReusedExchange [Reuses operator id: 62] +Output [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] -(57) ReusedExchange [Reuses operator id: 54] -Output [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +(67) CometSort +Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] +Arguments: [item_sk#43, d_date#44, web_sales#45, store_sales#46], [item_sk#43 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST] -(58) Sort [codegen id : 52] -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], false, 0 +(68) ColumnarToRow [codegen id : 34] +Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] -(59) Window -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#52], [item_sk#47], [d_date#48 ASC NULLS FIRST] +(69) Window +Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] +Arguments: [row_number() windowspecdefinition(item_sk#43, d_date#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#43], [d_date#44 ASC NULLS FIRST] -(60) Project [codegen id : 53] -Output [4]: [item_sk#47 AS item_sk#53, web_sales#49 AS web_sales#54, store_sales#50 AS store_sales#55, rk#52] -Input [5]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#52] +(70) Project [codegen id : 35] +Output [4]: [item_sk#43 AS item_sk#49, web_sales#45 AS web_sales#50, store_sales#46 AS store_sales#51, rk#48] +Input [5]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, rk#48] -(61) BroadcastExchange -Input [4]: [item_sk#53, web_sales#54, store_sales#55, rk#52] +(71) BroadcastExchange +Input [4]: [item_sk#49, web_sales#50, store_sales#51, rk#48] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(62) BroadcastHashJoin [codegen id : 54] -Left keys [1]: [item_sk#47] -Right keys [1]: [item_sk#53] +(72) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [item_sk#43] +Right keys [1]: [item_sk#49] Join type: Inner -Join condition: (rk#51 >= rk#52) - -(63) Project [codegen id : 54] -Output [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] -Input [9]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#51, item_sk#53, web_sales#54, store_sales#55, rk#52] - -(64) HashAggregate [codegen id : 54] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] -Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Functions [2]: [partial_max(web_sales#54), partial_max(store_sales#55)] -Aggregate Attributes [2]: [max#56, max#57] -Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] - -(65) HashAggregate [codegen id : 54] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] -Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Functions [2]: [max(web_sales#54), max(store_sales#55)] -Aggregate Attributes [2]: [max(web_sales#54)#60, max(store_sales#55)#61] -Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max(web_sales#54)#60 AS web_cumulative#62, max(store_sales#55)#61 AS store_cumulative#63] - -(66) Filter [codegen id : 54] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] -Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) - -(67) TakeOrderedAndProject -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] -Arguments: 100, [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] +Join condition: (rk#47 >= rk#48) + +(73) Project [codegen id : 36] +Output [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, web_sales#50, store_sales#51] +Input [9]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, rk#47, item_sk#49, web_sales#50, store_sales#51, rk#48] + +(74) HashAggregate [codegen id : 36] +Input [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, web_sales#50, store_sales#51] +Keys [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] +Functions [2]: [partial_max(web_sales#50), partial_max(store_sales#51)] +Aggregate Attributes [2]: [max#52, max#53] +Results [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, max#54, max#55] + +(75) HashAggregate [codegen id : 36] +Input [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, max#54, max#55] +Keys [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] +Functions [2]: [max(web_sales#50), max(store_sales#51)] +Aggregate Attributes [2]: [max(web_sales#50)#56, max(store_sales#51)#57] +Results [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, max(web_sales#50)#56 AS web_cumulative#58, max(store_sales#51)#57 AS store_cumulative#59] + +(76) Filter [codegen id : 36] +Input [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, web_cumulative#58, store_cumulative#59] +Condition : ((isnotnull(web_cumulative#58) AND isnotnull(store_cumulative#59)) AND (web_cumulative#58 > store_cumulative#59)) + +(77) TakeOrderedAndProject +Input [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, web_cumulative#58, store_cumulative#59] +Arguments: 100, [item_sk#43 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST], [item_sk#43, d_date#44, web_sales#45, store_sales#46, web_cumulative#58, store_cumulative#59] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (82) ++- * ColumnarToRow (81) + +- CometProject (80) + +- CometFilter (79) + +- CometScan parquet spark_catalog.default.date_dim (78) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#60] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] -Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) +(79) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#60] +Condition : (((isnotnull(d_month_seq#60) AND (d_month_seq#60 >= 1212)) AND (d_month_seq#60 <= 1223)) AND isnotnull(d_date_sk#5)) -(70) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +(80) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#60] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(71) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(72) BroadcastExchange +(82) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index 3109290dc7..ec52058fb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (54) + WholeStageCodegen (36) Filter [web_cumulative,store_cumulative] HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] @@ -7,118 +7,113 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (26) - Sort [item_sk,d_date] + WholeStageCodegen (17) + ColumnarToRow InputAdapter - Exchange [item_sk] #1 - WholeStageCodegen (25) - Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] - Filter [item_sk,item_sk] - SortMergeJoin [item_sk,d_date,item_sk,d_date] - InputAdapter - WholeStageCodegen (12) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - InputAdapter - Exchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] + CometSort [item_sk,d_date] + CometColumnarExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,item_sk] + CometSortMergeJoin [item_sk,d_date,item_sk,d_date] + CometSort [item_sk,d_date] + CometColumnarExchange [item_sk,d_date] #2 + CometHashAggregate [item_sk,d_date,sumws,sum,isEmpty] + CometColumnarExchange [item_sk,d_date,sumws] #3 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (3) + ColumnarToRow InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - Sort [ws_item_sk,d_date] - InputAdapter - Exchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - InputAdapter - Exchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + CometSort [ws_item_sk,d_date] + CometColumnarExchange [ws_item_sk] #4 + CometHashAggregate [ws_item_sk,d_date,sum] + CometColumnarExchange [ws_item_sk,d_date] #5 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + Project [item_sk,sumws,rk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Project [item_sk,sumws,rk] + Window [ws_item_sk,d_date] + WholeStageCodegen (6) + ColumnarToRow InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (8) - Sort [ws_item_sk,d_date] - InputAdapter - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - InputAdapter - WholeStageCodegen (24) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #8 - WholeStageCodegen (23) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - InputAdapter - Exchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (22) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] + CometSort [ws_item_sk,d_date] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date] + CometColumnarExchange [item_sk,d_date] #8 + CometHashAggregate [item_sk,d_date,sumss,sum,isEmpty] + CometColumnarExchange [item_sk,d_date,sumss] #9 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (11) + ColumnarToRow InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (16) - Sort [ss_item_sk,d_date] - InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (15) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - InputAdapter - Exchange [ss_item_sk,d_date] #11 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + CometSort [ss_item_sk,d_date] + CometColumnarExchange [ss_item_sk] #10 + CometHashAggregate [ss_item_sk,d_date,sum] + CometColumnarExchange [ss_item_sk,d_date] #11 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (15) + Project [item_sk,sumss,rk] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (21) - Project [item_sk,sumss,rk] + Window [ss_item_sk,d_date] + WholeStageCodegen (14) + ColumnarToRow InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (20) - Sort [ss_item_sk,d_date] - InputAdapter - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + CometSort [ss_item_sk,d_date] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 InputAdapter BroadcastExchange #13 - WholeStageCodegen (53) + WholeStageCodegen (35) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (52) - Sort [item_sk,d_date] + WholeStageCodegen (34) + ColumnarToRow InputAdapter - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometSort [item_sk,d_date] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index dff12158b6..966f99500b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * Filter (25) - : : +- Window (24) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- Window (33) - : +- * Sort (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) - +- BroadcastExchange (42) - +- * Project (41) - +- Window (40) - +- * Sort (39) - +- ReusedExchange (38) +TakeOrderedAndProject (47) ++- * Project (46) + +- * BroadcastHashJoin Inner BuildRight (45) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * ColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- CometHashAggregate (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.call_center (13) + : +- BroadcastExchange (36) + : +- * Project (35) + : +- Window (34) + : +- * ColumnarToRow (33) + : +- CometSort (32) + : +- ReusedExchange (31) + +- BroadcastExchange (44) + +- * Project (43) + +- Window (42) + +- * ColumnarToRow (41) + +- CometSort (40) + +- ReusedExchange (39) (unknown) Scan parquet spark_catalog.default.item @@ -89,7 +91,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(10) ReusedExchange [Reuses operator id: 51] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -137,125 +139,124 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#6))] Aggregate Attributes [1]: [sum#14] Results [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -(20) Exchange +(20) RowToColumnar Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) HashAggregate [codegen id : 5] +(21) CometColumnarExchange +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) CometHashAggregate Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#16] -Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS _w0#18] -(22) Exchange -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(23) CometColumnarExchange +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(23) Sort [codegen id : 6] -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 +(24) CometSort +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(24) Window -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(25) ColumnarToRow [codegen id : 5] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] + +(26) Window +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#18], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +(27) Filter [codegen id : 6] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(26) Window -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] - -(27) Filter [codegen id : 22] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(28) Project [codegen id : 22] -Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(29) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] - -(30) HashAggregate [codegen id : 12] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] -Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] - -(31) Exchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(32) Sort [codegen id : 13] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 - -(33) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) Project [codegen id : 14] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] - -(35) BroadcastExchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] - -(36) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] +(28) Window +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] +Arguments: [avg(_w0#17) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#19], [i_category#3, i_brand#2, cc_name#13, d_year#10] + +(29) Filter [codegen id : 19] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] +Condition : ((isnotnull(avg_monthly_sales#19) AND (avg_monthly_sales#19 > 0.000000)) AND CASE WHEN (avg_monthly_sales#19 > 0.000000) THEN ((abs((sum_sales#16 - avg_monthly_sales#19)) / avg_monthly_sales#19) > 0.1000000000000000) END) + +(30) Project [codegen id : 19] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] + +(31) ReusedExchange [Reuses operator id: 23] +Output [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] + +(32) CometSort +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16], [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, cc_name#22 ASC NULLS FIRST, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] + +(33) ColumnarToRow [codegen id : 11] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] + +(34) Window +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [rank(d_year#23, d_moy#24) windowspecdefinition(i_category#20, i_brand#21, cc_name#22, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#25], [i_category#20, i_brand#21, cc_name#22], [d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] + +(35) Project [codegen id : 12] +Output [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#16 AS sum_sales#26, rn#25] +Input [7]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16, rn#25] + +(36) BroadcastExchange +Input [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#26, rn#25] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 19] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] +Right keys [4]: [i_category#20, i_brand#21, cc_name#22, (rn#25 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +(38) Project [codegen id : 19] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#26] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, cc_name#22, sum_sales#26, rn#25] + +(39) ReusedExchange [Reuses operator id: 23] +Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] -(38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +(40) CometSort +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] +Arguments: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16], [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] -(39) Sort [codegen id : 20] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 +(41) ColumnarToRow [codegen id : 17] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] -(40) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +(42) Window +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] +Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] -(41) Project [codegen id : 21] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] +(43) Project [codegen id : 18] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#16 AS sum_sales#33, rn#32] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16, rn#32] -(42) BroadcastExchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] +(44) BroadcastExchange +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#33, rn#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] +(45) BroadcastHashJoin [codegen id : 19] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#32 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] -Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +(46) Project [codegen id : 19] +Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#26 AS psum#34, sum_sales#33 AS nsum#35] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#26, i_category#27, i_brand#28, cc_name#29, sum_sales#33, rn#32] -(45) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +(47) TakeOrderedAndProject +Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#34, nsum#35] +Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#34, nsum#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim @@ -265,15 +266,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(49) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(51) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index 56e33be9e4..174e9c66ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (19) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,18 +8,18 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (6) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometSort [i_category,i_brand,cc_name,d_year,d_moy] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + RowToColumnar WholeStageCodegen (4) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] @@ -57,25 +57,23 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (11) + ColumnarToRow InputAdapter - Exchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + CometSort [i_category,i_brand,cc_name,d_year,d_moy] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #7 + WholeStageCodegen (18) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - Sort [i_category,i_brand,cc_name,d_year,d_moy] + WholeStageCodegen (17) + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + CometSort [i_category,i_brand,cc_name,d_year,d_moy] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 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 d0c446ece2..06fede35c0 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,141 +1,159 @@ == Physical Plan == -TakeOrderedAndProject (137) -+- * HashAggregate (136) - +- Exchange (135) - +- * HashAggregate (134) - +- Union (133) - :- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- Union (65) - : :- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (11) - : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : :- * ColumnarToRow (8) - : : : : +- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : +- ReusedExchange (9) - : : +- BroadcastExchange (15) - : : +- * ColumnarToRow (14) - : : +- CometFilter (13) - : : +- CometScan parquet spark_catalog.default.store (12) - : :- * HashAggregate (40) - : : +- Exchange (39) - : : +- * HashAggregate (38) - : : +- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * ColumnarToRow (28) - : : : : +- CometUnion (27) - : : : : :- CometProject (23) - : : : : : +- CometFilter (22) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) - : : : : +- CometProject (26) - : : : : +- CometFilter (25) - : : : : +- CometScan parquet spark_catalog.default.catalog_returns (24) - : : : +- ReusedExchange (29) - : : +- BroadcastExchange (35) - : : +- * ColumnarToRow (34) - : : +- CometFilter (33) - : : +- CometScan parquet spark_catalog.default.catalog_page (32) - : +- * HashAggregate (64) - : +- Exchange (63) - : +- * HashAggregate (62) - : +- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Project (55) - : : +- * BroadcastHashJoin Inner BuildRight (54) - : : :- * ColumnarToRow (52) - : : : +- CometUnion (51) - : : : :- CometProject (43) - : : : : +- CometFilter (42) - : : : : +- CometScan parquet spark_catalog.default.web_sales (41) - : : : +- CometProject (50) - : : : +- CometBroadcastHashJoin (49) - : : : :- CometBroadcastExchange (45) - : : : : +- CometScan parquet spark_catalog.default.web_returns (44) - : : : +- CometProject (48) - : : : +- CometFilter (47) - : : : +- CometScan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_site (56) - :- * HashAggregate (100) - : +- Exchange (99) - : +- * HashAggregate (98) - : +- * HashAggregate (97) - : +- Exchange (96) - : +- * HashAggregate (95) - : +- Union (94) - : :- * HashAggregate (70) - : : +- ReusedExchange (69) - : :- * HashAggregate (72) - : : +- ReusedExchange (71) - : +- * HashAggregate (93) - : +- Exchange (92) - : +- * HashAggregate (91) - : +- * Project (90) - : +- * BroadcastHashJoin Inner BuildRight (89) - : :- * Project (87) - : : +- * BroadcastHashJoin Inner BuildRight (86) - : : :- * ColumnarToRow (84) - : : : +- CometUnion (83) - : : : :- CometProject (75) - : : : : +- CometFilter (74) - : : : : +- CometScan parquet spark_catalog.default.web_sales (73) - : : : +- CometProject (82) - : : : +- CometBroadcastHashJoin (81) - : : : :- CometBroadcastExchange (77) - : : : : +- CometScan parquet spark_catalog.default.web_returns (76) - : : : +- CometProject (80) - : : : +- CometFilter (79) - : : : +- CometScan parquet spark_catalog.default.web_sales (78) - : : +- ReusedExchange (85) - : +- ReusedExchange (88) - +- * HashAggregate (132) - +- Exchange (131) - +- * HashAggregate (130) - +- * HashAggregate (129) - +- Exchange (128) - +- * HashAggregate (127) - +- Union (126) - :- * HashAggregate (102) - : +- ReusedExchange (101) - :- * HashAggregate (104) - : +- ReusedExchange (103) - +- * HashAggregate (125) - +- Exchange (124) - +- * HashAggregate (123) - +- * Project (122) - +- * BroadcastHashJoin Inner BuildRight (121) - :- * Project (119) - : +- * BroadcastHashJoin Inner BuildRight (118) - : :- * ColumnarToRow (116) - : : +- CometUnion (115) - : : :- CometProject (107) - : : : +- CometFilter (106) - : : : +- CometScan parquet spark_catalog.default.web_sales (105) - : : +- CometProject (114) - : : +- CometBroadcastHashJoin (113) - : : :- CometBroadcastExchange (109) - : : : +- CometScan parquet spark_catalog.default.web_returns (108) - : : +- CometProject (112) - : : +- CometFilter (111) - : : +- CometScan parquet spark_catalog.default.web_sales (110) - : +- ReusedExchange (117) - +- ReusedExchange (120) +* ColumnarToRow (155) ++- CometTakeOrderedAndProject (154) + +- CometHashAggregate (153) + +- CometColumnarExchange (152) + +- CometHashAggregate (151) + +- CometUnion (150) + :- CometHashAggregate (75) + : +- CometColumnarExchange (74) + : +- RowToColumnar (73) + : +- * HashAggregate (72) + : +- Union (71) + : :- * HashAggregate (22) + : : +- * ColumnarToRow (21) + : : +- CometColumnarExchange (20) + : : +- RowToColumnar (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Project (11) + : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : :- * ColumnarToRow (8) + : : : : +- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : +- ReusedExchange (9) + : : +- BroadcastExchange (15) + : : +- * ColumnarToRow (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.store (12) + : :- * HashAggregate (44) + : : +- * ColumnarToRow (43) + : : +- CometColumnarExchange (42) + : : +- RowToColumnar (41) + : : +- * HashAggregate (40) + : : +- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * Project (33) + : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : :- * ColumnarToRow (30) + : : : : +- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (31) + : : +- BroadcastExchange (37) + : : +- * ColumnarToRow (36) + : : +- CometFilter (35) + : : +- CometScan parquet spark_catalog.default.catalog_page (34) + : +- * HashAggregate (70) + : +- * ColumnarToRow (69) + : +- CometColumnarExchange (68) + : +- RowToColumnar (67) + : +- * HashAggregate (66) + : +- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * ColumnarToRow (56) + : : : +- CometUnion (55) + : : : :- CometProject (47) + : : : : +- CometFilter (46) + : : : : +- CometScan parquet spark_catalog.default.web_sales (45) + : : : +- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometBroadcastExchange (49) + : : : : +- CometScan parquet spark_catalog.default.web_returns (48) + : : : +- CometProject (52) + : : : +- CometFilter (51) + : : : +- CometScan parquet spark_catalog.default.web_sales (50) + : : +- ReusedExchange (57) + : +- BroadcastExchange (63) + : +- * ColumnarToRow (62) + : +- CometFilter (61) + : +- CometScan parquet spark_catalog.default.web_site (60) + :- CometHashAggregate (112) + : +- CometColumnarExchange (111) + : +- CometHashAggregate (110) + : +- CometHashAggregate (109) + : +- CometColumnarExchange (108) + : +- RowToColumnar (107) + : +- * HashAggregate (106) + : +- Union (105) + : :- * HashAggregate (78) + : : +- * ColumnarToRow (77) + : : +- ReusedExchange (76) + : :- * HashAggregate (81) + : : +- * ColumnarToRow (80) + : : +- ReusedExchange (79) + : +- * HashAggregate (104) + : +- * ColumnarToRow (103) + : +- CometColumnarExchange (102) + : +- RowToColumnar (101) + : +- * HashAggregate (100) + : +- * Project (99) + : +- * BroadcastHashJoin Inner BuildRight (98) + : :- * Project (96) + : : +- * BroadcastHashJoin Inner BuildRight (95) + : : :- * ColumnarToRow (93) + : : : +- CometUnion (92) + : : : :- CometProject (84) + : : : : +- CometFilter (83) + : : : : +- CometScan parquet spark_catalog.default.web_sales (82) + : : : +- CometProject (91) + : : : +- CometBroadcastHashJoin (90) + : : : :- CometBroadcastExchange (86) + : : : : +- CometScan parquet spark_catalog.default.web_returns (85) + : : : +- CometProject (89) + : : : +- CometFilter (88) + : : : +- CometScan parquet spark_catalog.default.web_sales (87) + : : +- ReusedExchange (94) + : +- ReusedExchange (97) + +- CometHashAggregate (149) + +- CometColumnarExchange (148) + +- CometHashAggregate (147) + +- CometHashAggregate (146) + +- CometColumnarExchange (145) + +- RowToColumnar (144) + +- * HashAggregate (143) + +- Union (142) + :- * HashAggregate (115) + : +- * ColumnarToRow (114) + : +- ReusedExchange (113) + :- * HashAggregate (118) + : +- * ColumnarToRow (117) + : +- ReusedExchange (116) + +- * HashAggregate (141) + +- * ColumnarToRow (140) + +- CometColumnarExchange (139) + +- RowToColumnar (138) + +- * HashAggregate (137) + +- * Project (136) + +- * BroadcastHashJoin Inner BuildRight (135) + :- * Project (133) + : +- * BroadcastHashJoin Inner BuildRight (132) + : :- * ColumnarToRow (130) + : : +- CometUnion (129) + : : :- CometProject (121) + : : : +- CometFilter (120) + : : : +- CometScan parquet spark_catalog.default.web_sales (119) + : : +- CometProject (128) + : : +- CometBroadcastHashJoin (127) + : : :- CometBroadcastExchange (123) + : : : +- CometScan parquet spark_catalog.default.web_returns (122) + : : +- CometProject (126) + : : +- CometFilter (125) + : : +- CometScan parquet spark_catalog.default.web_sales (124) + : +- ReusedExchange (131) + +- ReusedExchange (134) (unknown) Scan parquet spark_catalog.default.store_sales @@ -177,7 +195,7 @@ Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_a (8) ColumnarToRow [codegen id : 3] Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -(9) ReusedExchange [Reuses operator id: 142] +(9) ReusedExchange [Reuses operator id: 160] Output [1]: [d_date_sk#22] (10) BroadcastHashJoin [codegen id : 3] @@ -225,11 +243,17 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -(19) Exchange +(19) RowToColumnar Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(20) HashAggregate [codegen id : 4] +(20) CometColumnarExchange +Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(21) ColumnarToRow [codegen id : 4] +Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] + +(22) HashAggregate [codegen id : 4] Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] Keys [1]: [s_store_id#24] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] @@ -244,11 +268,11 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(22) CometFilter +(24) CometFilter Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Condition : isnotnull(cs_catalog_page_sk#42) -(23) CometProject +(25) CometProject Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Arguments: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52], [cs_catalog_page_sk#42 AS page_sk#47, cs_sold_date_sk#45 AS date_sk#48, cs_ext_sales_price#43 AS sales_price#49, cs_net_profit#44 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] @@ -260,31 +284,31 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(25) CometFilter +(27) CometFilter Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Condition : isnotnull(cr_catalog_page_sk#53) -(26) CometProject +(28) CometProject Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Arguments: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62], [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] -(27) CometUnion +(29) CometUnion Child 0 Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62] -(28) ColumnarToRow [codegen id : 7] +(30) ColumnarToRow [codegen id : 7] Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] -(29) ReusedExchange [Reuses operator id: 142] +(31) ReusedExchange [Reuses operator id: 160] Output [1]: [d_date_sk#63] -(30) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 7] Left keys [1]: [date_sk#48] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(31) Project [codegen id : 7] +(33) Project [codegen id : 7] Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] @@ -295,39 +319,45 @@ Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(33) CometFilter +(35) CometFilter Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Condition : isnotnull(cp_catalog_page_sk#64) -(34) ColumnarToRow [codegen id : 6] +(36) ColumnarToRow [codegen id : 6] Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -(35) BroadcastExchange +(37) BroadcastExchange Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(36) BroadcastHashJoin [codegen id : 7] +(38) BroadcastHashJoin [codegen id : 7] Left keys [1]: [page_sk#47] Right keys [1]: [cp_catalog_page_sk#64] Join type: Inner Join condition: None -(37) Project [codegen id : 7] +(39) Project [codegen id : 7] Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#65] -(38) HashAggregate [codegen id : 7] +(40) HashAggregate [codegen id : 7] Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] Aggregate Attributes [4]: [sum#66, sum#67, sum#68, sum#69] Results [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -(39) Exchange +(41) RowToColumnar +Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] + +(42) CometColumnarExchange +Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(43) ColumnarToRow [codegen id : 8] Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(40) HashAggregate [codegen id : 8] +(44) HashAggregate [codegen id : 8] Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -342,11 +372,11 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(42) CometFilter +(46) CometFilter Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Condition : isnotnull(ws_web_site_sk#83) -(43) CometProject +(47) CometProject Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] @@ -357,7 +387,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] ReadSchema: struct -(45) CometBroadcastExchange +(49) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] @@ -368,40 +398,40 @@ 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 -(47) CometFilter +(51) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(48) CometProject +(52) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(49) CometBroadcastHashJoin +(53) CometBroadcastHashJoin Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(50) CometProject +(54) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -(51) CometUnion +(55) CometUnion Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] -(52) ColumnarToRow [codegen id : 11] +(56) ColumnarToRow [codegen id : 11] Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(53) ReusedExchange [Reuses operator id: 142] +(57) ReusedExchange [Reuses operator id: 160] Output [1]: [d_date_sk#109] -(54) BroadcastHashJoin [codegen id : 11] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(55) Project [codegen id : 11] +(59) Project [codegen id : 11] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -412,80 +442,93 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(57) CometFilter +(61) CometFilter Input [2]: [web_site_sk#110, web_site_id#111] Condition : isnotnull(web_site_sk#110) -(58) ColumnarToRow [codegen id : 10] +(62) ColumnarToRow [codegen id : 10] Input [2]: [web_site_sk#110, web_site_id#111] -(59) BroadcastExchange +(63) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#111] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(60) BroadcastHashJoin [codegen id : 11] +(64) BroadcastHashJoin [codegen id : 11] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(61) Project [codegen id : 11] +(65) Project [codegen id : 11] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(62) HashAggregate [codegen id : 11] +(66) HashAggregate [codegen id : 11] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -(63) Exchange +(67) RowToColumnar +Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] + +(68) CometColumnarExchange Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(64) HashAggregate [codegen id : 12] +(69) ColumnarToRow [codegen id : 12] +Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] + +(70) HashAggregate [codegen id : 12] Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] -(65) Union +(71) Union -(66) HashAggregate [codegen id : 13] +(72) HashAggregate [codegen id : 13] Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] Keys [2]: [channel#37, id#38] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -(67) Exchange +(73) RowToColumnar +Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] + +(74) CometColumnarExchange Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(68) HashAggregate [codegen id : 14] +(75) CometHashAggregate Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] -Results [5]: [channel#37, id#38, cast(sum(sales#39)#141 as decimal(37,2)) AS sales#144, cast(sum(returns#40)#142 as decimal(37,2)) AS returns#145, cast(sum(profit#41)#143 as decimal(38,2)) AS profit#146] -(69) ReusedExchange [Reuses operator id: 19] -Output [5]: [s_store_id#24, sum#147, sum#148, sum#149, sum#150] +(76) ReusedExchange [Reuses operator id: 20] +Output [5]: [s_store_id#24, sum#141, sum#142, sum#143, sum#144] -(70) HashAggregate [codegen id : 18] -Input [5]: [s_store_id#24, sum#147, sum#148, sum#149, sum#150] +(77) ColumnarToRow [codegen id : 17] +Input [5]: [s_store_id#24, sum#141, sum#142, sum#143, sum#144] + +(78) HashAggregate [codegen id : 17] +Input [5]: [s_store_id#24, sum#141, sum#142, sum#143, sum#144] Keys [1]: [s_store_id#24] 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))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] -(71) ReusedExchange [Reuses operator id: 39] -Output [5]: [cp_catalog_page_id#65, sum#151, sum#152, sum#153, sum#154] +(79) ReusedExchange [Reuses operator id: 42] +Output [5]: [cp_catalog_page_id#65, sum#145, sum#146, sum#147, sum#148] + +(80) ColumnarToRow [codegen id : 21] +Input [5]: [cp_catalog_page_id#65, sum#145, sum#146, sum#147, sum#148] -(72) HashAggregate [codegen id : 22] -Input [5]: [cp_catalog_page_id#65, sum#151, sum#152, sum#153, sum#154] +(81) HashAggregate [codegen id : 21] +Input [5]: [cp_catalog_page_id#65, sum#145, sum#146, sum#147, sum#148] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] @@ -495,15 +538,15 @@ Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_pag Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#155)] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#149)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(74) CometFilter +(83) CometFilter Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Condition : isnotnull(ws_web_site_sk#83) -(75) CometProject +(84) CometProject Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] @@ -511,10 +554,10 @@ Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_am Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#155)] +PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#149)] ReadSchema: struct -(77) CometBroadcastExchange +(86) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] @@ -525,127 +568,136 @@ 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 -(79) CometFilter +(88) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(80) CometProject +(89) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(81) CometBroadcastHashJoin +(90) CometBroadcastHashJoin Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(82) CometProject +(91) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -(83) CometUnion +(92) CometUnion Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] -(84) ColumnarToRow [codegen id : 25] +(93) ColumnarToRow [codegen id : 24] Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(85) ReusedExchange [Reuses operator id: 142] +(94) ReusedExchange [Reuses operator id: 160] Output [1]: [d_date_sk#109] -(86) BroadcastHashJoin [codegen id : 25] +(95) BroadcastHashJoin [codegen id : 24] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(87) Project [codegen id : 25] +(96) Project [codegen id : 24] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] -(88) ReusedExchange [Reuses operator id: 59] +(97) ReusedExchange [Reuses operator id: 63] Output [2]: [web_site_sk#110, web_site_id#111] -(89) BroadcastHashJoin [codegen id : 25] +(98) BroadcastHashJoin [codegen id : 24] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(90) Project [codegen id : 25] +(99) Project [codegen id : 24] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(91) HashAggregate [codegen id : 25] +(100) HashAggregate [codegen id : 24] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#156, sum#157, sum#158, sum#159] -Results [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] +Aggregate Attributes [4]: [sum#150, sum#151, sum#152, sum#153] +Results [5]: [web_site_id#111, sum#154, sum#155, sum#156, sum#157] + +(101) RowToColumnar +Input [5]: [web_site_id#111, sum#154, sum#155, sum#156, sum#157] + +(102) CometColumnarExchange +Input [5]: [web_site_id#111, sum#154, sum#155, sum#156, sum#157] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(92) Exchange -Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(103) ColumnarToRow [codegen id : 25] +Input [5]: [web_site_id#111, sum#154, sum#155, sum#156, sum#157] -(93) HashAggregate [codegen id : 26] -Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] +(104) HashAggregate [codegen id : 25] +Input [5]: [web_site_id#111, sum#154, sum#155, sum#156, sum#157] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] -(94) Union +(105) Union -(95) HashAggregate [codegen id : 27] +(106) HashAggregate [codegen id : 26] Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] Keys [2]: [channel#37, id#38] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -(96) Exchange +(107) RowToColumnar Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(97) HashAggregate [codegen id : 28] +(108) CometColumnarExchange +Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(109) CometHashAggregate Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] -Results [4]: [channel#37, sum(sales#39)#141 AS sales#164, sum(returns#40)#142 AS returns#165, sum(profit#41)#143 AS profit#166] -(98) HashAggregate [codegen id : 28] -Input [4]: [channel#37, sales#164, returns#165, profit#166] +(110) CometHashAggregate +Input [4]: [channel#37, sales#158, returns#159, profit#160] Keys [1]: [channel#37] -Functions [3]: [partial_sum(sales#164), partial_sum(returns#165), partial_sum(profit#166)] -Aggregate Attributes [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] -Results [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] +Functions [3]: [partial_sum(sales#158), partial_sum(returns#159), partial_sum(profit#160)] -(99) Exchange -Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] -Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(111) CometColumnarExchange +Input [7]: [channel#37, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166] +Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(100) HashAggregate [codegen id : 29] -Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] +(112) CometHashAggregate +Input [7]: [channel#37, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166] Keys [1]: [channel#37] -Functions [3]: [sum(sales#164), sum(returns#165), sum(profit#166)] -Aggregate Attributes [3]: [sum(sales#164)#179, sum(returns#165)#180, sum(profit#166)#181] -Results [5]: [channel#37, null AS id#182, sum(sales#164)#179 AS sum(sales)#183, sum(returns#165)#180 AS sum(returns)#184, sum(profit#166)#181 AS sum(profit)#185] +Functions [3]: [sum(sales#158), sum(returns#159), sum(profit#160)] + +(113) ReusedExchange [Reuses operator id: 20] +Output [5]: [s_store_id#24, sum#167, sum#168, sum#169, sum#170] -(101) ReusedExchange [Reuses operator id: 19] -Output [5]: [s_store_id#24, sum#186, sum#187, sum#188, sum#189] +(114) ColumnarToRow [codegen id : 30] +Input [5]: [s_store_id#24, sum#167, sum#168, sum#169, sum#170] -(102) HashAggregate [codegen id : 33] -Input [5]: [s_store_id#24, sum#186, sum#187, sum#188, sum#189] +(115) HashAggregate [codegen id : 30] +Input [5]: [s_store_id#24, sum#167, sum#168, sum#169, sum#170] Keys [1]: [s_store_id#24] 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))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] -(103) ReusedExchange [Reuses operator id: 39] -Output [5]: [cp_catalog_page_id#65, sum#190, sum#191, sum#192, sum#193] +(116) ReusedExchange [Reuses operator id: 42] +Output [5]: [cp_catalog_page_id#65, sum#171, sum#172, sum#173, sum#174] -(104) HashAggregate [codegen id : 37] -Input [5]: [cp_catalog_page_id#65, sum#190, sum#191, sum#192, sum#193] +(117) ColumnarToRow [codegen id : 34] +Input [5]: [cp_catalog_page_id#65, sum#171, sum#172, sum#173, sum#174] + +(118) HashAggregate [codegen id : 34] +Input [5]: [cp_catalog_page_id#65, sum#171, sum#172, sum#173, sum#174] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] @@ -655,15 +707,15 @@ Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_pag Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#194)] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#175)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(106) CometFilter +(120) CometFilter Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Condition : isnotnull(ws_web_site_sk#83) -(107) CometProject +(121) CometProject Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] @@ -671,10 +723,10 @@ Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_am Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#194)] +PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#175)] ReadSchema: struct -(109) CometBroadcastExchange +(123) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] @@ -685,184 +737,189 @@ 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 -(111) CometFilter +(125) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(112) CometProject +(126) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(113) CometBroadcastHashJoin +(127) CometBroadcastHashJoin Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(114) CometProject +(128) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -(115) CometUnion +(129) CometUnion Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] -(116) ColumnarToRow [codegen id : 40] +(130) ColumnarToRow [codegen id : 37] Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(117) ReusedExchange [Reuses operator id: 142] +(131) ReusedExchange [Reuses operator id: 160] Output [1]: [d_date_sk#109] -(118) BroadcastHashJoin [codegen id : 40] +(132) BroadcastHashJoin [codegen id : 37] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(119) Project [codegen id : 40] +(133) Project [codegen id : 37] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] -(120) ReusedExchange [Reuses operator id: 59] +(134) ReusedExchange [Reuses operator id: 63] Output [2]: [web_site_sk#110, web_site_id#111] -(121) BroadcastHashJoin [codegen id : 40] +(135) BroadcastHashJoin [codegen id : 37] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(122) Project [codegen id : 40] +(136) Project [codegen id : 37] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(123) HashAggregate [codegen id : 40] +(137) HashAggregate [codegen id : 37] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#195, sum#196, sum#197, sum#198] -Results [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] +Aggregate Attributes [4]: [sum#176, sum#177, sum#178, sum#179] +Results [5]: [web_site_id#111, sum#180, sum#181, sum#182, sum#183] + +(138) RowToColumnar +Input [5]: [web_site_id#111, sum#180, sum#181, sum#182, sum#183] + +(139) CometColumnarExchange +Input [5]: [web_site_id#111, sum#180, sum#181, sum#182, sum#183] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(124) Exchange -Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(140) ColumnarToRow [codegen id : 38] +Input [5]: [web_site_id#111, sum#180, sum#181, sum#182, sum#183] -(125) HashAggregate [codegen id : 41] -Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] +(141) HashAggregate [codegen id : 38] +Input [5]: [web_site_id#111, sum#180, sum#181, sum#182, sum#183] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] -(126) Union +(142) Union -(127) HashAggregate [codegen id : 42] +(143) HashAggregate [codegen id : 39] Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] Keys [2]: [channel#37, id#38] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -(128) Exchange +(144) RowToColumnar Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(129) HashAggregate [codegen id : 43] +(145) CometColumnarExchange +Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(146) CometHashAggregate Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] -Results [3]: [sum(sales#39)#141 AS sales#164, sum(returns#40)#142 AS returns#165, sum(profit#41)#143 AS profit#166] -(130) HashAggregate [codegen id : 43] -Input [3]: [sales#164, returns#165, profit#166] +(147) CometHashAggregate +Input [3]: [sales#158, returns#159, profit#160] Keys: [] -Functions [3]: [partial_sum(sales#164), partial_sum(returns#165), partial_sum(profit#166)] -Aggregate Attributes [6]: [sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208] -Results [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] +Functions [3]: [partial_sum(sales#158), partial_sum(returns#159), partial_sum(profit#160)] -(131) Exchange -Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(148) CometColumnarExchange +Input [6]: [sum#184, isEmpty#185, sum#186, isEmpty#187, sum#188, isEmpty#189] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(132) HashAggregate [codegen id : 44] -Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] +(149) CometHashAggregate +Input [6]: [sum#184, isEmpty#185, sum#186, isEmpty#187, sum#188, isEmpty#189] Keys: [] -Functions [3]: [sum(sales#164), sum(returns#165), sum(profit#166)] -Aggregate Attributes [3]: [sum(sales#164)#215, sum(returns#165)#216, sum(profit#166)#217] -Results [5]: [null AS channel#218, null AS id#219, sum(sales#164)#215 AS sum(sales)#220, sum(returns#165)#216 AS sum(returns)#221, sum(profit#166)#217 AS sum(profit)#222] +Functions [3]: [sum(sales#158), sum(returns#159), sum(profit#160)] -(133) Union +(150) CometUnion +Child 0 Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] +Child 1 Input [5]: [channel#37, id#193, sum(sales)#194, sum(returns)#195, sum(profit)#196] +Child 2 Input [5]: [channel#197, id#198, sum(sales)#199, sum(returns)#200, sum(profit)#201] -(134) HashAggregate [codegen id : 45] -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +(151) CometHashAggregate +Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] +Keys [5]: [channel#37, id#38, sales#190, returns#191, profit#192] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -(135) Exchange -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Arguments: hashpartitioning(channel#37, id#38, sales#144, returns#145, profit#146, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(152) CometColumnarExchange +Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] +Arguments: hashpartitioning(channel#37, id#38, sales#190, returns#191, profit#192, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(136) HashAggregate [codegen id : 46] -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +(153) CometHashAggregate +Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] +Keys [5]: [channel#37, id#38, sales#190, returns#191, profit#192] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -(137) TakeOrderedAndProject -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#144, returns#145, profit#146] +(154) CometTakeOrderedAndProject +Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#37 ASC NULLS FIRST,id#38 ASC NULLS FIRST], output=[channel#37,id#38,sales#190,returns#191,profit#192]), 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#190, returns#191, profit#192] + +(155) ColumnarToRow [codegen id : 40] +Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (142) -+- * ColumnarToRow (141) - +- CometProject (140) - +- CometFilter (139) - +- CometScan parquet spark_catalog.default.date_dim (138) +BroadcastExchange (160) ++- * ColumnarToRow (159) + +- CometProject (158) + +- CometFilter (157) + +- CometScan parquet spark_catalog.default.date_dim (156) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#223] +Output [2]: [d_date_sk#22, d_date#202] 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 -(139) CometFilter -Input [2]: [d_date_sk#22, d_date#223] -Condition : (((isnotnull(d_date#223) AND (d_date#223 >= 1998-08-04)) AND (d_date#223 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +(157) CometFilter +Input [2]: [d_date_sk#22, d_date#202] +Condition : (((isnotnull(d_date#202) AND (d_date#202 >= 1998-08-04)) AND (d_date#202 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(140) CometProject -Input [2]: [d_date_sk#22, d_date#223] +(158) CometProject +Input [2]: [d_date_sk#22, d_date#202] Arguments: [d_date_sk#22], [d_date_sk#22] -(141) ColumnarToRow [codegen id : 1] +(159) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(142) BroadcastExchange +(160) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 48 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 -Subquery:7 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:7 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:8 Hosting operator id = 76 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:8 Hosting operator id = 85 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 -Subquery:9 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:9 Hosting operator id = 119 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:10 Hosting operator id = 108 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 122 Hosting Expression = wr_returned_date_sk#98 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 ff62cb5c24..d24e3bae4b 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,24 +1,24 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (46) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (45) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (14) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [channel,id] #2 - WholeStageCodegen (13) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (4) - 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] - InputAdapter - Exchange [s_store_id] #3 +WholeStageCodegen (40) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (13) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + RowToColumnar WholeStageCodegen (3) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,s_store_id] @@ -52,10 +52,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (8) - 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] - InputAdapter - Exchange [cp_catalog_page_id] #6 + WholeStageCodegen (8) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #6 + RowToColumnar WholeStageCodegen (7) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] @@ -82,10 +84,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter CometFilter [cp_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (12) - 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] - InputAdapter - Exchange [web_site_id] #8 + WholeStageCodegen (12) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #8 + RowToColumnar WholeStageCodegen (11) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] @@ -116,103 +120,105 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter CometFilter [web_site_sk] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (29) - 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] - InputAdapter - Exchange [channel] #11 - WholeStageCodegen (28) - 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] + CometHashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [channel] #11 + CometHashAggregate [channel,sales,returns,profit] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [channel,id] #12 + RowToColumnar + WholeStageCodegen (26) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [channel,id] #12 - WholeStageCodegen (27) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (18) - 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] - InputAdapter - ReusedExchange [s_store_id,sum,sum,sum,sum] #3 - WholeStageCodegen (22) - 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] - InputAdapter - ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 - WholeStageCodegen (26) - 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] - InputAdapter - Exchange [web_site_id] #13 - WholeStageCodegen (25) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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,ws_item_sk,ws_order_number] - CometBroadcastExchange #14 - CometScan 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_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 + Union + WholeStageCodegen (17) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [s_store_id,sum,sum,sum,sum] #3 + WholeStageCodegen (21) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 + WholeStageCodegen (25) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #13 + RowToColumnar + WholeStageCodegen (24) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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,ws_item_sk,ws_order_number] + CometBroadcastExchange #14 + CometScan 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_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] InputAdapter - ReusedExchange [web_site_sk,web_site_id] #10 - WholeStageCodegen (44) - 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] - InputAdapter - Exchange #15 - WholeStageCodegen (43) - 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] + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [web_site_sk,web_site_id] #10 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange #15 + CometHashAggregate [sales,returns,profit] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [channel,id] #16 + RowToColumnar + WholeStageCodegen (39) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [channel,id] #16 - WholeStageCodegen (42) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (33) - 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] - InputAdapter - ReusedExchange [s_store_id,sum,sum,sum,sum] #3 - WholeStageCodegen (37) - 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] - InputAdapter - ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 - WholeStageCodegen (41) - 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] - InputAdapter - Exchange [web_site_id] #17 - WholeStageCodegen (40) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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,ws_item_sk,ws_order_number] - CometBroadcastExchange #18 - CometScan 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_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 + Union + WholeStageCodegen (30) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [s_store_id,sum,sum,sum,sum] #3 + WholeStageCodegen (34) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 + WholeStageCodegen (38) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #17 + RowToColumnar + WholeStageCodegen (37) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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,ws_item_sk,ws_order_number] + CometBroadcastExchange #18 + CometScan 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_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] InputAdapter - ReusedExchange [web_site_sk,web_site_id] #10 + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [web_site_sk,web_site_id] #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index fcaa84890b..ffd243c6af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -1,43 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Filter (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer_address (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometFilter (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.item (22) +* ColumnarToRow (41) ++- CometTakeOrderedAndProject (40) + +- CometFilter (39) + +- CometHashAggregate (38) + +- CometColumnarExchange (37) + +- RowToColumnar (36) + +- * HashAggregate (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer_address (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store_sales (10) + : +- ReusedExchange (16) + +- BroadcastExchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * ColumnarToRow (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) + +- CometHashAggregate (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.item (22) (unknown) Scan parquet spark_catalog.default.customer_address @@ -111,7 +113,7 @@ Join condition: None Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -(16) ReusedExchange [Reuses operator id: 44] +(16) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#9] (17) BroadcastHashJoin [codegen id : 6] @@ -154,9 +156,9 @@ Input [2]: [i_current_price#13, i_category#14] Keys [1]: [i_category#14] Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] -(25) CometExchange +(25) CometColumnarExchange Input [3]: [i_category#14, sum#15, count#16] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (26) CometHashAggregate Input [3]: [i_category#14, sum#15, count#16] @@ -205,97 +207,101 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] Results [2]: [ca_state#2, count#19] -(36) Exchange +(36) RowToColumnar Input [2]: [ca_state#2, count#19] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(37) HashAggregate [codegen id : 7] +(37) CometColumnarExchange +Input [2]: [ca_state#2, count#19] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(38) CometHashAggregate Input [2]: [ca_state#2, count#19] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#20] -Results [3]: [ca_state#2 AS state#21, count(1)#20 AS cnt#22, ca_state#2] -(38) Filter [codegen id : 7] -Input [3]: [state#21, cnt#22, ca_state#2] -Condition : (cnt#22 >= 10) +(39) CometFilter +Input [3]: [state#20, cnt#21, ca_state#2] +Condition : (cnt#21 >= 10) + +(40) CometTakeOrderedAndProject +Input [3]: [state#20, cnt#21, ca_state#2] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#21 ASC NULLS FIRST,ca_state#2 ASC NULLS FIRST], output=[state#20,cnt#21]), 100, [cnt#21 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#20, cnt#21] -(39) TakeOrderedAndProject -Input [3]: [state#21, cnt#22, ca_state#2] -Arguments: 100, [cnt#22 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#21, cnt#22] +(41) ColumnarToRow [codegen id : 7] +Input [2]: [state#20, cnt#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#23] +Output [2]: [d_date_sk#9, d_month_seq#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#23] -Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-subquery#24, [id=#25])) AND isnotnull(d_date_sk#9)) +(43) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#22] +Condition : ((isnotnull(d_month_seq#22) AND (d_month_seq#22 = Subquery scalar-subquery#23, [id=#24])) AND isnotnull(d_date_sk#9)) -(42) CometProject -Input [2]: [d_date_sk#9, d_month_seq#23] +(44) CometProject +Input [2]: [d_date_sk#9, d_month_seq#22] Arguments: [d_date_sk#9], [d_date_sk#9] -(43) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(44) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#24, [id=#25] -* ColumnarToRow (51) -+- CometHashAggregate (50) - +- CometExchange (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +Subquery:2 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#23, [id=#24] +* ColumnarToRow (53) ++- CometHashAggregate (52) + +- CometColumnarExchange (51) + +- CometHashAggregate (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] +Output [3]: [d_month_seq#25, d_year#26, d_moy#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(46) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) +(48) CometFilter +Input [3]: [d_month_seq#25, d_year#26, d_moy#27] +Condition : (((isnotnull(d_year#26) AND isnotnull(d_moy#27)) AND (d_year#26 = 2000)) AND (d_moy#27 = 1)) -(47) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] +(49) CometProject +Input [3]: [d_month_seq#25, d_year#26, d_moy#27] +Arguments: [d_month_seq#25], [d_month_seq#25] -(48) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +(50) CometHashAggregate +Input [1]: [d_month_seq#25] +Keys [1]: [d_month_seq#25] Functions: [] -(49) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(51) CometColumnarExchange +Input [1]: [d_month_seq#25] +Arguments: hashpartitioning(d_month_seq#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(50) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +(52) CometHashAggregate +Input [1]: [d_month_seq#25] +Keys [1]: [d_month_seq#25] Functions: [] -(51) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] +(53) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#25] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index d2126126d5..2efd9c5510 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -1,74 +1,76 @@ -TakeOrderedAndProject [cnt,ca_state,state] - WholeStageCodegen (7) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - InputAdapter - Exchange [ca_state] #1 - WholeStageCodegen (6) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] +WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [cnt,ca_state,state] + CometFilter [cnt] + CometHashAggregate [ca_state,count] + CometColumnarExchange [ca_state] #1 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ca_state,ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ca_state,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [ca_state,c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_current_addr_sk,c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometFilter [ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometColumnarExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + Project [i_item_sk] + BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow - InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [avg(i_current_price)] - CometHashAggregate [i_category,sum,count] - CometExchange [i_category] #8 - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_current_price,i_category] + CometFilter [i_current_price,i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [avg(i_current_price)] + CometHashAggregate [i_category,sum,count] + CometColumnarExchange [i_category] #8 + CometHashAggregate [i_category,i_current_price] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 1429e39daa..fdfaa1e2cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -1,183 +1,189 @@ == Physical Plan == -* Sort (179) -+- Exchange (178) - +- * Project (177) - +- * SortMergeJoin Inner (176) - :- * Sort (108) - : +- Exchange (107) - : +- * HashAggregate (106) - : +- * HashAggregate (105) - : +- * Project (104) - : +- * BroadcastHashJoin Inner BuildRight (103) - : :- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * Project (94) - : : : +- * BroadcastHashJoin Inner BuildRight (93) - : : : :- * Project (88) - : : : : +- * BroadcastHashJoin Inner BuildRight (87) - : : : : :- * Project (85) - : : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : : :- * Project (79) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : :- * Project (76) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : : : : : :- * Project (70) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (69) - : : : : : : : : :- * Project (64) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : : : : : : :- * Project (61) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : : : : : : : :- * Project (55) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : : : : : : : :- * Project (52) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : : : : : : : : : : :- * Project (46) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : : : : : : : : : : :- * Project (40) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : : : : : : : : : : : : :- * Project (34) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : : : : : : : : : : : :- * ColumnarToRow (31) - : : : : : : : : : : : : : : : : +- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- ReusedExchange (32) - : : : : : : : : : : : : : : +- BroadcastExchange (38) - : : : : : : : : : : : : : : +- * ColumnarToRow (37) - : : : : : : : : : : : : : : +- CometFilter (36) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (35) - : : : : : : : : : : : : : +- BroadcastExchange (44) - : : : : : : : : : : : : : +- * ColumnarToRow (43) - : : : : : : : : : : : : : +- CometFilter (42) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) - : : : : : : : : : : : : +- BroadcastExchange (50) - : : : : : : : : : : : : +- * ColumnarToRow (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (53) - : : : : : : : : : : +- BroadcastExchange (59) - : : : : : : : : : : +- * ColumnarToRow (58) - : : : : : : : : : : +- CometFilter (57) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (56) - : : : : : : : : : +- ReusedExchange (62) - : : : : : : : : +- BroadcastExchange (68) - : : : : : : : : +- * ColumnarToRow (67) - : : : : : : : : +- CometFilter (66) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (65) - : : : : : : : +- BroadcastExchange (74) - : : : : : : : +- * ColumnarToRow (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (77) - : : : : : +- BroadcastExchange (83) - : : : : : +- * ColumnarToRow (82) - : : : : : +- CometFilter (81) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (80) - : : : : +- ReusedExchange (86) - : : : +- BroadcastExchange (92) - : : : +- * ColumnarToRow (91) - : : : +- CometFilter (90) - : : : +- CometScan parquet spark_catalog.default.income_band (89) - : : +- ReusedExchange (95) - : +- BroadcastExchange (102) - : +- * ColumnarToRow (101) - : +- CometProject (100) - : +- CometFilter (99) - : +- CometScan parquet spark_catalog.default.item (98) - +- * Sort (175) - +- Exchange (174) - +- * HashAggregate (173) - +- * HashAggregate (172) - +- * Project (171) - +- * BroadcastHashJoin Inner BuildRight (170) - :- * Project (168) - : +- * BroadcastHashJoin Inner BuildRight (167) - : :- * Project (165) - : : +- * BroadcastHashJoin Inner BuildRight (164) - : : :- * Project (162) - : : : +- * BroadcastHashJoin Inner BuildRight (161) - : : : :- * Project (159) - : : : : +- * BroadcastHashJoin Inner BuildRight (158) - : : : : :- * Project (156) - : : : : : +- * BroadcastHashJoin Inner BuildRight (155) - : : : : : :- * Project (153) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (152) - : : : : : : :- * Project (150) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (149) - : : : : : : : :- * Project (147) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (146) - : : : : : : : : :- * Project (144) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (143) - : : : : : : : : : :- * Project (141) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (140) - : : : : : : : : : : :- * Project (138) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (137) - : : : : : : : : : : : :- * Project (135) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (134) - : : : : : : : : : : : : :- * Project (132) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (131) - : : : : : : : : : : : : : :- * Project (129) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (128) - : : : : : : : : : : : : : : :- * ColumnarToRow (126) - : : : : : : : : : : : : : : : +- CometProject (125) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (124) - : : : : : : : : : : : : : : : :- CometSort (118) - : : : : : : : : : : : : : : : : +- CometExchange (117) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (115) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (111) - : : : : : : : : : : : : : : : : : +- CometFilter (110) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (109) - : : : : : : : : : : : : : : : : +- CometProject (114) - : : : : : : : : : : : : : : : : +- CometFilter (113) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (112) - : : : : : : : : : : : : : : : +- CometSort (123) - : : : : : : : : : : : : : : : +- CometProject (122) - : : : : : : : : : : : : : : : +- CometFilter (121) - : : : : : : : : : : : : : : : +- CometHashAggregate (120) - : : : : : : : : : : : : : : : +- ReusedExchange (119) - : : : : : : : : : : : : : : +- ReusedExchange (127) - : : : : : : : : : : : : : +- ReusedExchange (130) - : : : : : : : : : : : : +- ReusedExchange (133) - : : : : : : : : : : : +- ReusedExchange (136) - : : : : : : : : : : +- ReusedExchange (139) - : : : : : : : : : +- ReusedExchange (142) - : : : : : : : : +- ReusedExchange (145) - : : : : : : : +- ReusedExchange (148) - : : : : : : +- ReusedExchange (151) - : : : : : +- ReusedExchange (154) - : : : : +- ReusedExchange (157) - : : : +- ReusedExchange (160) - : : +- ReusedExchange (163) - : +- ReusedExchange (166) - +- ReusedExchange (169) +* ColumnarToRow (185) ++- CometSort (184) + +- CometColumnarExchange (183) + +- RowToColumnar (182) + +- * Project (181) + +- * SortMergeJoin Inner (180) + :- * ColumnarToRow (110) + : +- CometSort (109) + : +- CometColumnarExchange (108) + : +- RowToColumnar (107) + : +- * HashAggregate (106) + : +- * HashAggregate (105) + : +- * Project (104) + : +- * BroadcastHashJoin Inner BuildRight (103) + : :- * Project (97) + : : +- * BroadcastHashJoin Inner BuildRight (96) + : : :- * Project (94) + : : : +- * BroadcastHashJoin Inner BuildRight (93) + : : : :- * Project (88) + : : : : +- * BroadcastHashJoin Inner BuildRight (87) + : : : : :- * Project (85) + : : : : : +- * BroadcastHashJoin Inner BuildRight (84) + : : : : : :- * Project (79) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) + : : : : : : :- * Project (76) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) + : : : : : : : :- * Project (70) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (69) + : : : : : : : : :- * Project (64) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : : : : : : : :- * Project (61) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : : : : : : : :- * Project (55) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : : : : : : : :- * Project (52) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : : : : : : : : : :- * Project (46) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : : : : : : : : : : : :- * Project (40) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (39) + : : : : : : : : : : : : : : :- * Project (34) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : : : : : : : : : : : : :- * ColumnarToRow (31) + : : : : : : : : : : : : : : : : +- CometProject (30) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) + : : : : : : : : : : : : : : : : :- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- CometSort (28) + : : : : : : : : : : : : : : : : +- CometProject (27) + : : : : : : : : : : : : : : : : +- CometFilter (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (24) + : : : : : : : : : : : : : : : : +- CometHashAggregate (23) + : : : : : : : : : : : : : : : : +- CometProject (22) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) + : : : : : : : : : : : : : : : : :- CometSort (15) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (14) + : : : : : : : : : : : : : : : : : +- CometProject (13) + : : : : : : : : : : : : : : : : : +- CometFilter (12) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : : : : : : : : : : : +- CometSort (20) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (19) + : : : : : : : : : : : : : : : : +- CometProject (18) + : : : : : : : : : : : : : : : : +- CometFilter (17) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : : : : : : : : : : : : : : : +- ReusedExchange (32) + : : : : : : : : : : : : : : +- BroadcastExchange (38) + : : : : : : : : : : : : : : +- * ColumnarToRow (37) + : : : : : : : : : : : : : : +- CometFilter (36) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (35) + : : : : : : : : : : : : : +- BroadcastExchange (44) + : : : : : : : : : : : : : +- * ColumnarToRow (43) + : : : : : : : : : : : : : +- CometFilter (42) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) + : : : : : : : : : : : : +- BroadcastExchange (50) + : : : : : : : : : : : : +- * ColumnarToRow (49) + : : : : : : : : : : : : +- CometFilter (48) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (47) + : : : : : : : : : : : +- ReusedExchange (53) + : : : : : : : : : : +- BroadcastExchange (59) + : : : : : : : : : : +- * ColumnarToRow (58) + : : : : : : : : : : +- CometFilter (57) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (56) + : : : : : : : : : +- ReusedExchange (62) + : : : : : : : : +- BroadcastExchange (68) + : : : : : : : : +- * ColumnarToRow (67) + : : : : : : : : +- CometFilter (66) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (65) + : : : : : : : +- BroadcastExchange (74) + : : : : : : : +- * ColumnarToRow (73) + : : : : : : : +- CometFilter (72) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (71) + : : : : : : +- ReusedExchange (77) + : : : : : +- BroadcastExchange (83) + : : : : : +- * ColumnarToRow (82) + : : : : : +- CometFilter (81) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (80) + : : : : +- ReusedExchange (86) + : : : +- BroadcastExchange (92) + : : : +- * ColumnarToRow (91) + : : : +- CometFilter (90) + : : : +- CometScan parquet spark_catalog.default.income_band (89) + : : +- ReusedExchange (95) + : +- BroadcastExchange (102) + : +- * ColumnarToRow (101) + : +- CometProject (100) + : +- CometFilter (99) + : +- CometScan parquet spark_catalog.default.item (98) + +- * ColumnarToRow (179) + +- CometSort (178) + +- CometColumnarExchange (177) + +- RowToColumnar (176) + +- * HashAggregate (175) + +- * HashAggregate (174) + +- * Project (173) + +- * BroadcastHashJoin Inner BuildRight (172) + :- * Project (170) + : +- * BroadcastHashJoin Inner BuildRight (169) + : :- * Project (167) + : : +- * BroadcastHashJoin Inner BuildRight (166) + : : :- * Project (164) + : : : +- * BroadcastHashJoin Inner BuildRight (163) + : : : :- * Project (161) + : : : : +- * BroadcastHashJoin Inner BuildRight (160) + : : : : :- * Project (158) + : : : : : +- * BroadcastHashJoin Inner BuildRight (157) + : : : : : :- * Project (155) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) + : : : : : : :- * Project (152) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) + : : : : : : : :- * Project (149) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) + : : : : : : : : :- * Project (146) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) + : : : : : : : : : :- * Project (143) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) + : : : : : : : : : : :- * Project (140) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) + : : : : : : : : : : : :- * Project (137) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) + : : : : : : : : : : : : :- * Project (134) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) + : : : : : : : : : : : : : :- * Project (131) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) + : : : : : : : : : : : : : : :- * ColumnarToRow (128) + : : : : : : : : : : : : : : : +- CometProject (127) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (126) + : : : : : : : : : : : : : : : :- CometSort (120) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (119) + : : : : : : : : : : : : : : : : +- CometProject (118) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) + : : : : : : : : : : : : : : : : : +- CometFilter (112) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) + : : : : : : : : : : : : : : : : +- CometProject (116) + : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) + : : : : : : : : : : : : : : : +- CometSort (125) + : : : : : : : : : : : : : : : +- CometProject (124) + : : : : : : : : : : : : : : : +- CometFilter (123) + : : : : : : : : : : : : : : : +- CometHashAggregate (122) + : : : : : : : : : : : : : : : +- ReusedExchange (121) + : : : : : : : : : : : : : : +- ReusedExchange (129) + : : : : : : : : : : : : : +- ReusedExchange (132) + : : : : : : : : : : : : +- ReusedExchange (135) + : : : : : : : : : : : +- ReusedExchange (138) + : : : : : : : : : : +- ReusedExchange (141) + : : : : : : : : : +- ReusedExchange (144) + : : : : : : : : +- ReusedExchange (147) + : : : : : : : +- ReusedExchange (150) + : : : : : : +- ReusedExchange (153) + : : : : : +- ReusedExchange (156) + : : : : +- ReusedExchange (159) + : : : +- ReusedExchange (162) + : : +- ReusedExchange (165) + : +- ReusedExchange (168) + +- ReusedExchange (171) (unknown) Scan parquet spark_catalog.default.store_sales @@ -220,9 +226,9 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) CometExchange +(9) CometColumnarExchange Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] (10) CometSort Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] @@ -243,9 +249,9 @@ Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(14) CometExchange +(14) CometColumnarExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (15) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] @@ -266,9 +272,9 @@ Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(19) CometExchange +(19) CometColumnarExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (20) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] @@ -288,9 +294,9 @@ Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reverse Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(24) CometExchange +(24) CometColumnarExchange Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] (25) CometHashAggregate Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] @@ -321,7 +327,7 @@ Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_add (31) ColumnarToRow [codegen id : 16] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(32) ReusedExchange [Reuses operator id: 183] +(32) ReusedExchange [Reuses operator id: 189] Output [2]: [d_date_sk#32, d_year#33] (33) BroadcastHashJoin [codegen id : 16] @@ -669,13 +675,19 @@ Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledV Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#9))#81, sum(UnscaledValue(ss_list_price#10))#82, sum(UnscaledValue(ss_coupon_amt#11))#83] Results [17]: [i_product_name#71 AS product_name#84, i_item_sk#68 AS item_sk#85, s_store_name#35 AS store_name#86, s_zip#36 AS store_zip#87, ca_street_number#57 AS b_street_number#88, ca_street_name#58 AS b_streen_name#89, ca_city#59 AS b_city#90, ca_zip#60 AS b_zip#91, ca_street_number#62 AS c_street_number#92, ca_street_name#63 AS c_street_name#93, ca_city#64 AS c_city#94, ca_zip#65 AS c_zip#95, d_year#33 AS syear#96, count(1)#80 AS cnt#97, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#81,17,2) AS s1#98, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#82,17,2) AS s2#99, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#83,17,2) AS s3#100] -(107) Exchange +(107) RowToColumnar Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(108) Sort [codegen id : 17] +(108) CometColumnarExchange +Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] +Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(109) CometSort +Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] +Arguments: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100], [item_sk#85 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, store_zip#87 ASC NULLS FIRST] + +(110) ColumnarToRow [codegen id : 17] Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: [item_sk#85 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, store_zip#87 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_sales Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] @@ -685,11 +697,11 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#112), dynamicpruningexpression(ss_s PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(110) CometFilter +(112) CometFilter Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Condition : (((((((isnotnull(ss_item_sk#101) AND isnotnull(ss_ticket_number#108)) AND isnotnull(ss_store_sk#106)) AND isnotnull(ss_customer_sk#102)) AND isnotnull(ss_cdemo_sk#103)) AND isnotnull(ss_promo_sk#107)) AND isnotnull(ss_hdemo_sk#104)) AND isnotnull(ss_addr_sk#105)) -(111) CometBroadcastExchange +(113) CometBroadcastExchange Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] @@ -700,305 +712,317 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(113) CometFilter +(115) CometFilter Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] Condition : (isnotnull(sr_item_sk#114) AND isnotnull(sr_ticket_number#115)) -(114) CometProject +(116) CometProject Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] Arguments: [sr_item_sk#114, sr_ticket_number#115], [sr_item_sk#114, sr_ticket_number#115] -(115) CometBroadcastHashJoin +(117) CometBroadcastHashJoin Left output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Right output [2]: [sr_item_sk#114, sr_ticket_number#115] Arguments: [ss_item_sk#101, ss_ticket_number#108], [sr_item_sk#114, sr_ticket_number#115], Inner -(116) CometProject +(118) CometProject Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, sr_item_sk#114, sr_ticket_number#115] Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -(117) CometExchange +(119) CometColumnarExchange Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] +Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(118) CometSort +(120) CometSort Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101 ASC NULLS FIRST] -(119) ReusedExchange [Reuses operator id: 24] +(121) ReusedExchange [Reuses operator id: 24] Output [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] -(120) CometHashAggregate +(122) CometHashAggregate Input [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] Keys [1]: [cs_item_sk#117] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#121)), sum(((cr_refunded_cash#122 + cr_reversed_charge#123) + cr_store_credit#124))] -(121) CometFilter +(123) CometFilter Input [3]: [cs_item_sk#117, sale#30, refund#31] Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(122) CometProject +(124) CometProject Input [3]: [cs_item_sk#117, sale#30, refund#31] Arguments: [cs_item_sk#117], [cs_item_sk#117] -(123) CometSort +(125) CometSort Input [1]: [cs_item_sk#117] Arguments: [cs_item_sk#117], [cs_item_sk#117 ASC NULLS FIRST] -(124) CometSortMergeJoin +(126) CometSortMergeJoin Left output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] Right output [1]: [cs_item_sk#117] Arguments: [ss_item_sk#101], [cs_item_sk#117], Inner -(125) CometProject +(127) CometProject Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, cs_item_sk#117] Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -(126) ColumnarToRow [codegen id : 33] +(128) ColumnarToRow [codegen id : 33] Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -(127) ReusedExchange [Reuses operator id: 187] +(129) ReusedExchange [Reuses operator id: 193] Output [2]: [d_date_sk#125, d_year#126] -(128) BroadcastHashJoin [codegen id : 33] +(130) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_sold_date_sk#112] Right keys [1]: [d_date_sk#125] Join type: Inner Join condition: None -(129) Project [codegen id : 33] +(131) Project [codegen id : 33] Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126] Input [13]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, d_date_sk#125, d_year#126] -(130) ReusedExchange [Reuses operator id: 38] +(132) ReusedExchange [Reuses operator id: 38] Output [3]: [s_store_sk#127, s_store_name#128, s_zip#129] -(131) BroadcastHashJoin [codegen id : 33] +(133) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_store_sk#106] Right keys [1]: [s_store_sk#127] Join type: Inner Join condition: None -(132) Project [codegen id : 33] +(134) Project [codegen id : 33] Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129] Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_sk#127, s_store_name#128, s_zip#129] -(133) ReusedExchange [Reuses operator id: 44] +(135) ReusedExchange [Reuses operator id: 44] Output [6]: [c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -(134) BroadcastHashJoin [codegen id : 33] +(136) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_customer_sk#102] Right keys [1]: [c_customer_sk#130] Join type: Inner Join condition: None -(135) Project [codegen id : 33] +(137) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] Input [18]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -(136) ReusedExchange [Reuses operator id: 50] +(138) ReusedExchange [Reuses operator id: 50] Output [2]: [d_date_sk#136, d_year#137] -(137) BroadcastHashJoin [codegen id : 33] +(139) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_first_sales_date_sk#135] Right keys [1]: [d_date_sk#136] Join type: Inner Join condition: None -(138) Project [codegen id : 33] +(140) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135, d_date_sk#136, d_year#137] -(139) ReusedExchange [Reuses operator id: 50] +(141) ReusedExchange [Reuses operator id: 50] Output [2]: [d_date_sk#138, d_year#139] -(140) BroadcastHashJoin [codegen id : 33] +(142) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_first_shipto_date_sk#134] Right keys [1]: [d_date_sk#138] Join type: Inner Join condition: None -(141) Project [codegen id : 33] +(143) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137, d_date_sk#138, d_year#139] -(142) ReusedExchange [Reuses operator id: 59] +(144) ReusedExchange [Reuses operator id: 59] Output [2]: [cd_demo_sk#140, cd_marital_status#141] -(143) BroadcastHashJoin [codegen id : 33] +(145) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_cdemo_sk#103] Right keys [1]: [cd_demo_sk#140] Join type: Inner Join condition: None -(144) Project [codegen id : 33] +(146) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141] Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_demo_sk#140, cd_marital_status#141] -(145) ReusedExchange [Reuses operator id: 59] +(147) ReusedExchange [Reuses operator id: 59] Output [2]: [cd_demo_sk#142, cd_marital_status#143] -(146) BroadcastHashJoin [codegen id : 33] +(148) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_cdemo_sk#131] Right keys [1]: [cd_demo_sk#142] Join type: Inner Join condition: NOT (cd_marital_status#141 = cd_marital_status#143) -(147) Project [codegen id : 33] +(149) Project [codegen id : 33] Output [14]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [18]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141, cd_demo_sk#142, cd_marital_status#143] -(148) ReusedExchange [Reuses operator id: 68] +(150) ReusedExchange [Reuses operator id: 68] Output [1]: [p_promo_sk#144] -(149) BroadcastHashJoin [codegen id : 33] +(151) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_promo_sk#107] Right keys [1]: [p_promo_sk#144] Join type: Inner Join condition: None -(150) Project [codegen id : 33] +(152) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, p_promo_sk#144] -(151) ReusedExchange [Reuses operator id: 74] +(153) ReusedExchange [Reuses operator id: 74] Output [2]: [hd_demo_sk#145, hd_income_band_sk#146] -(152) BroadcastHashJoin [codegen id : 33] +(154) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_hdemo_sk#104] Right keys [1]: [hd_demo_sk#145] Join type: Inner Join condition: None -(153) Project [codegen id : 33] +(155) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146] Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_demo_sk#145, hd_income_band_sk#146] -(154) ReusedExchange [Reuses operator id: 74] +(156) ReusedExchange [Reuses operator id: 74] Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] -(155) BroadcastHashJoin [codegen id : 33] +(157) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_hdemo_sk#132] Right keys [1]: [hd_demo_sk#147] Join type: Inner Join condition: None -(156) Project [codegen id : 33] +(158) Project [codegen id : 33] Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148] Input [15]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_demo_sk#147, hd_income_band_sk#148] -(157) ReusedExchange [Reuses operator id: 83] +(159) ReusedExchange [Reuses operator id: 83] Output [5]: [ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -(158) BroadcastHashJoin [codegen id : 33] +(160) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_addr_sk#105] Right keys [1]: [ca_address_sk#149] Join type: Inner Join condition: None -(159) Project [codegen id : 33] +(161) Project [codegen id : 33] Output [16]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] Input [18]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -(160) ReusedExchange [Reuses operator id: 83] +(162) ReusedExchange [Reuses operator id: 83] Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(161) BroadcastHashJoin [codegen id : 33] +(163) BroadcastHashJoin [codegen id : 33] Left keys [1]: [c_current_addr_sk#133] Right keys [1]: [ca_address_sk#154] Join type: Inner Join condition: None -(162) Project [codegen id : 33] +(164) Project [codegen id : 33] Output [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [21]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(163) ReusedExchange [Reuses operator id: 92] +(165) ReusedExchange [Reuses operator id: 92] Output [1]: [ib_income_band_sk#159] -(164) BroadcastHashJoin [codegen id : 33] +(166) BroadcastHashJoin [codegen id : 33] Left keys [1]: [hd_income_band_sk#146] Right keys [1]: [ib_income_band_sk#159] Join type: Inner Join condition: None -(165) Project [codegen id : 33] +(167) Project [codegen id : 33] Output [18]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [20]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#159] -(166) ReusedExchange [Reuses operator id: 92] +(168) ReusedExchange [Reuses operator id: 92] Output [1]: [ib_income_band_sk#160] -(167) BroadcastHashJoin [codegen id : 33] +(169) BroadcastHashJoin [codegen id : 33] Left keys [1]: [hd_income_band_sk#148] Right keys [1]: [ib_income_band_sk#160] Join type: Inner Join condition: None -(168) Project [codegen id : 33] +(170) Project [codegen id : 33] Output [17]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#160] -(169) ReusedExchange [Reuses operator id: 102] +(171) ReusedExchange [Reuses operator id: 102] Output [2]: [i_item_sk#161, i_product_name#162] -(170) BroadcastHashJoin [codegen id : 33] +(172) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_item_sk#101] Right keys [1]: [i_item_sk#161] Join type: Inner Join condition: None -(171) Project [codegen id : 33] +(173) Project [codegen id : 33] Output [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -(172) HashAggregate [codegen id : 33] +(174) HashAggregate [codegen id : 33] Input [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#109)), partial_sum(UnscaledValue(ss_list_price#110)), partial_sum(UnscaledValue(ss_coupon_amt#111))] Aggregate Attributes [4]: [count#72, sum#163, sum#164, sum#165] Results [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] -(173) HashAggregate [codegen id : 33] +(175) HashAggregate [codegen id : 33] Input [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#109)), sum(UnscaledValue(ss_list_price#110)), sum(UnscaledValue(ss_coupon_amt#111))] Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#109))#81, sum(UnscaledValue(ss_list_price#110))#82, sum(UnscaledValue(ss_coupon_amt#111))#83] Results [8]: [i_item_sk#161 AS item_sk#169, s_store_name#128 AS store_name#170, s_zip#129 AS store_zip#171, d_year#126 AS syear#172, count(1)#80 AS cnt#173, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#109))#81,17,2) AS s1#174, MakeDecimal(sum(UnscaledValue(ss_list_price#110))#82,17,2) AS s2#175, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#111))#83,17,2) AS s3#176] -(174) Exchange +(176) RowToColumnar +Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] + +(177) CometColumnarExchange +Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] +Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] + +(178) CometSort Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176], [item_sk#169 ASC NULLS FIRST, store_name#170 ASC NULLS FIRST, store_zip#171 ASC NULLS FIRST] -(175) Sort [codegen id : 34] +(179) ColumnarToRow [codegen id : 34] Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: [item_sk#169 ASC NULLS FIRST, store_name#170 ASC NULLS FIRST, store_zip#171 ASC NULLS FIRST], false, 0 -(176) SortMergeJoin [codegen id : 35] +(180) SortMergeJoin [codegen id : 35] Left keys [3]: [item_sk#85, store_name#86, store_zip#87] Right keys [3]: [item_sk#169, store_name#170, store_zip#171] Join type: Inner Join condition: (cnt#173 <= cnt#97) -(177) Project [codegen id : 35] +(181) Project [codegen id : 35] Output [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] Input [25]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -(178) Exchange +(182) RowToColumnar +Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] + +(183) CometColumnarExchange +Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] +Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, s1#98 ASC NULLS FIRST, s1#174 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] + +(184) CometSort Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, s1#98 ASC NULLS FIRST, s1#174 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] +Arguments: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173], [product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, s1#98 ASC NULLS FIRST, s1#174 ASC NULLS FIRST] -(179) Sort [codegen id : 36] +(185) ColumnarToRow [codegen id : 36] Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: [product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, s1#98 ASC NULLS FIRST, s1#174 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (183) -+- * ColumnarToRow (182) - +- CometFilter (181) - +- CometScan parquet spark_catalog.default.date_dim (180) +BroadcastExchange (189) ++- * ColumnarToRow (188) + +- CometFilter (187) + +- CometScan parquet spark_catalog.default.date_dim (186) (unknown) Scan parquet spark_catalog.default.date_dim @@ -1008,22 +1032,22 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(181) CometFilter +(187) CometFilter Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(182) ColumnarToRow [codegen id : 1] +(188) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] -(183) BroadcastExchange +(189) BroadcastExchange Input [2]: [d_date_sk#32, d_year#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 109 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 -BroadcastExchange (187) -+- * ColumnarToRow (186) - +- CometFilter (185) - +- CometScan parquet spark_catalog.default.date_dim (184) +Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 +BroadcastExchange (193) ++- * ColumnarToRow (192) + +- CometFilter (191) + +- CometScan parquet spark_catalog.default.date_dim (190) (unknown) Scan parquet spark_catalog.default.date_dim @@ -1033,14 +1057,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(185) CometFilter +(191) CometFilter Input [2]: [d_date_sk#125, d_year#126] Condition : ((isnotnull(d_year#126) AND (d_year#126 = 2000)) AND isnotnull(d_date_sk#125)) -(186) ColumnarToRow [codegen id : 1] +(192) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#125, d_year#126] -(187) BroadcastExchange +(193) BroadcastExchange Input [2]: [d_date_sk#125, d_year#126] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index e2accd7418..f297b41473 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -1,254 +1,260 @@ WholeStageCodegen (36) - Sort [product_name,store_name,cnt,s1,s1] + ColumnarToRow InputAdapter - Exchange [product_name,store_name,cnt,s1,s1] #1 - WholeStageCodegen (35) - Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] - InputAdapter - WholeStageCodegen (17) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (16) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometExchange [ss_item_sk] #3 + CometSort [product_name,store_name,cnt,s1,s1] + CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 + RowToColumnar + WholeStageCodegen (35) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (17) + ColumnarToRow + InputAdapter + CometSort [item_sk,store_name,store_zip] + CometColumnarExchange [item_sk,store_name,store_zip] #2 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometSortMergeJoin [ss_item_sk,cs_item_sk] + CometSort [ss_item_sk] + CometColumnarExchange [ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #4 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [sale,refund] + CometHashAggregate [cs_item_sk,sum,sum,isEmpty] + CometColumnarExchange [cs_item_sk] #6 + CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number] + CometColumnarExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + BroadcastExchange #10 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) + BroadcastExchange #11 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (4) + BroadcastExchange #12 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (6) + BroadcastExchange #13 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + BroadcastExchange #14 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - BroadcastExchange #14 - WholeStageCodegen (9) + BroadcastExchange #15 + WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) + BroadcastExchange #16 + WholeStageCodegen (13) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (13) + BroadcastExchange #17 + WholeStageCodegen (15) ColumnarToRow InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (15) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - InputAdapter - WholeStageCodegen (34) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (33) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometExchange [ss_item_sk] #19 + CometProject [i_item_sk,i_product_name] + CometFilter [i_current_price,i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (34) + ColumnarToRow + InputAdapter + CometSort [item_sk,store_name,store_zip] + CometColumnarExchange [item_sk,store_name,store_zip] #18 + RowToColumnar + WholeStageCodegen (33) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + CometSortMergeJoin [ss_item_sk,cs_item_sk] + CometSort [ss_item_sk] + CometColumnarExchange [ss_item_sk] #19 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #20 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [sale,refund] + CometHashAggregate [cs_item_sk,sum,sum,isEmpty] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #21 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 InputAdapter - ReusedExchange [d_date_sk,d_year] #21 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [p_promo_sk] #13 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [p_promo_sk] #13 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [ib_income_band_sk] #16 InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + ReusedExchange [i_item_sk,i_product_name] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index 68be5c8723..5c22de6679 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -1,71 +1,73 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Filter (66) - +- Window (65) - +- * Sort (64) - +- Exchange (63) - +- Union (62) - :- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.item (13) - :- * HashAggregate (26) - : +- Exchange (25) - : +- * HashAggregate (24) - : +- * HashAggregate (23) - : +- ReusedExchange (22) - :- * HashAggregate (31) - : +- Exchange (30) - : +- * HashAggregate (29) - : +- * HashAggregate (28) - : +- ReusedExchange (27) - :- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * HashAggregate (33) - : +- ReusedExchange (32) - :- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * HashAggregate (38) - : +- ReusedExchange (37) - :- * HashAggregate (46) - : +- Exchange (45) - : +- * HashAggregate (44) - : +- * HashAggregate (43) - : +- ReusedExchange (42) - :- * HashAggregate (51) - : +- Exchange (50) - : +- * HashAggregate (49) - : +- * HashAggregate (48) - : +- ReusedExchange (47) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * HashAggregate (53) - : +- ReusedExchange (52) - +- * HashAggregate (61) - +- Exchange (60) - +- * HashAggregate (59) - +- * HashAggregate (58) - +- ReusedExchange (57) +TakeOrderedAndProject (69) ++- * Filter (68) + +- Window (67) + +- * ColumnarToRow (66) + +- CometSort (65) + +- CometColumnarExchange (64) + +- CometUnion (63) + :- CometHashAggregate (22) + : +- CometColumnarExchange (21) + : +- RowToColumnar (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.item (13) + :- CometHashAggregate (27) + : +- CometColumnarExchange (26) + : +- CometHashAggregate (25) + : +- CometHashAggregate (24) + : +- ReusedExchange (23) + :- CometHashAggregate (32) + : +- CometColumnarExchange (31) + : +- CometHashAggregate (30) + : +- CometHashAggregate (29) + : +- ReusedExchange (28) + :- CometHashAggregate (37) + : +- CometColumnarExchange (36) + : +- CometHashAggregate (35) + : +- CometHashAggregate (34) + : +- ReusedExchange (33) + :- CometHashAggregate (42) + : +- CometColumnarExchange (41) + : +- CometHashAggregate (40) + : +- CometHashAggregate (39) + : +- ReusedExchange (38) + :- CometHashAggregate (47) + : +- CometColumnarExchange (46) + : +- CometHashAggregate (45) + : +- CometHashAggregate (44) + : +- ReusedExchange (43) + :- CometHashAggregate (52) + : +- CometColumnarExchange (51) + : +- CometHashAggregate (50) + : +- CometHashAggregate (49) + : +- ReusedExchange (48) + :- CometHashAggregate (57) + : +- CometColumnarExchange (56) + : +- CometHashAggregate (55) + : +- CometHashAggregate (54) + : +- ReusedExchange (53) + +- CometHashAggregate (62) + +- CometColumnarExchange (61) + +- CometHashAggregate (60) + +- CometHashAggregate (59) + +- ReusedExchange (58) (unknown) Scan parquet spark_catalog.default.store_sales @@ -83,7 +85,7 @@ Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 72] +(4) ReusedExchange [Reuses operator id: 74] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (5) BroadcastHashJoin [codegen id : 4] @@ -159,292 +161,257 @@ Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as d Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] -(20) Exchange +(20) RowToColumnar Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) HashAggregate [codegen id : 5] +(21) CometColumnarExchange +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(22) CometHashAggregate Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23] -(22) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25] +(23) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#22, isEmpty#23] -(23) HashAggregate [codegen id : 10] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25] +(24) CometHashAggregate +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#22, isEmpty#23] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(24) HashAggregate [codegen id : 10] -Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sumsales#26] +(25) CometHashAggregate +Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sumsales#24] Keys [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#27, isEmpty#28] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] +Functions [1]: [partial_sum(sumsales#24)] -(25) Exchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(26) CometColumnarExchange +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#25, isEmpty#26] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) HashAggregate [codegen id : 11] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] +(27) CometHashAggregate +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#25, isEmpty#26] Keys [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#31] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, null AS s_store_id#32, sum(sumsales#26)#31 AS sumsales#33] +Functions [1]: [sum(sumsales#24)] -(27) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35] +(28) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#27, isEmpty#28] -(28) HashAggregate [codegen id : 16] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35] +(29) CometHashAggregate +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#27, isEmpty#28] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(29) HashAggregate [codegen id : 16] -Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sumsales#26] +(30) CometHashAggregate +Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sumsales#24] Keys [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] +Functions [1]: [partial_sum(sumsales#24)] -(30) Exchange -Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(31) CometColumnarExchange +Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#29, isEmpty#30] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(31) HashAggregate [codegen id : 17] -Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] +(32) CometHashAggregate +Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#29, isEmpty#30] Keys [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#40] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, null AS d_moy#41, null AS s_store_id#42, sum(sumsales#26)#40 AS sumsales#43] +Functions [1]: [sum(sumsales#24)] -(32) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45] +(33) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#31, isEmpty#32] -(33) HashAggregate [codegen id : 22] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45] +(34) CometHashAggregate +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#31, isEmpty#32] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(34) HashAggregate [codegen id : 22] -Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sumsales#26] +(35) CometHashAggregate +Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sumsales#24] Keys [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] +Functions [1]: [partial_sum(sumsales#24)] -(35) Exchange -Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(36) CometColumnarExchange +Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#33, isEmpty#34] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(36) HashAggregate [codegen id : 23] -Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] +(37) CometHashAggregate +Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#33, isEmpty#34] Keys [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#50] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, null AS d_qoy#51, null AS d_moy#52, null AS s_store_id#53, sum(sumsales#26)#50 AS sumsales#54] +Functions [1]: [sum(sumsales#24)] -(37) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56] +(38) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#35, isEmpty#36] -(38) HashAggregate [codegen id : 28] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56] +(39) CometHashAggregate +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#35, isEmpty#36] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(39) HashAggregate [codegen id : 28] -Input [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sumsales#26] +(40) CometHashAggregate +Input [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sumsales#24] Keys [4]: [i_category#16, i_class#15, i_brand#14, i_product_name#17] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] +Functions [1]: [partial_sum(sumsales#24)] -(40) Exchange -Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(41) CometColumnarExchange +Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#37, isEmpty#38] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(41) HashAggregate [codegen id : 29] -Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] +(42) CometHashAggregate +Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#37, isEmpty#38] Keys [4]: [i_category#16, i_class#15, i_brand#14, i_product_name#17] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#61] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, null AS d_year#62, null AS d_qoy#63, null AS d_moy#64, null AS s_store_id#65, sum(sumsales#26)#61 AS sumsales#66] +Functions [1]: [sum(sumsales#24)] -(42) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68] +(43) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#39, isEmpty#40] -(43) HashAggregate [codegen id : 34] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68] +(44) CometHashAggregate +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#39, isEmpty#40] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [4]: [i_category#16, i_class#15, i_brand#14, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(44) HashAggregate [codegen id : 34] -Input [4]: [i_category#16, i_class#15, i_brand#14, sumsales#26] +(45) CometHashAggregate +Input [4]: [i_category#16, i_class#15, i_brand#14, sumsales#24] Keys [3]: [i_category#16, i_class#15, i_brand#14] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#69, isEmpty#70] -Results [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] +Functions [1]: [partial_sum(sumsales#24)] -(45) Exchange -Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(46) CometColumnarExchange +Input [5]: [i_category#16, i_class#15, i_brand#14, sum#41, isEmpty#42] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(46) HashAggregate [codegen id : 35] -Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] +(47) CometHashAggregate +Input [5]: [i_category#16, i_class#15, i_brand#14, sum#41, isEmpty#42] Keys [3]: [i_category#16, i_class#15, i_brand#14] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#73] -Results [9]: [i_category#16, i_class#15, i_brand#14, null AS i_product_name#74, null AS d_year#75, null AS d_qoy#76, null AS d_moy#77, null AS s_store_id#78, sum(sumsales#26)#73 AS sumsales#79] +Functions [1]: [sum(sumsales#24)] -(47) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81] +(48) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#43, isEmpty#44] -(48) HashAggregate [codegen id : 40] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81] +(49) CometHashAggregate +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#43, isEmpty#44] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [3]: [i_category#16, i_class#15, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(49) HashAggregate [codegen id : 40] -Input [3]: [i_category#16, i_class#15, sumsales#26] +(50) CometHashAggregate +Input [3]: [i_category#16, i_class#15, sumsales#24] Keys [2]: [i_category#16, i_class#15] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#82, isEmpty#83] -Results [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] +Functions [1]: [partial_sum(sumsales#24)] -(50) Exchange -Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] -Arguments: hashpartitioning(i_category#16, i_class#15, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(51) CometColumnarExchange +Input [4]: [i_category#16, i_class#15, sum#45, isEmpty#46] +Arguments: hashpartitioning(i_category#16, i_class#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(51) HashAggregate [codegen id : 41] -Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] +(52) CometHashAggregate +Input [4]: [i_category#16, i_class#15, sum#45, isEmpty#46] Keys [2]: [i_category#16, i_class#15] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#86] -Results [9]: [i_category#16, i_class#15, null AS i_brand#87, null AS i_product_name#88, null AS d_year#89, null AS d_qoy#90, null AS d_moy#91, null AS s_store_id#92, sum(sumsales#26)#86 AS sumsales#93] +Functions [1]: [sum(sumsales#24)] -(52) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95] +(53) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#47, isEmpty#48] -(53) HashAggregate [codegen id : 46] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95] +(54) CometHashAggregate +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#47, isEmpty#48] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [2]: [i_category#16, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(54) HashAggregate [codegen id : 46] -Input [2]: [i_category#16, sumsales#26] +(55) CometHashAggregate +Input [2]: [i_category#16, sumsales#24] Keys [1]: [i_category#16] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#96, isEmpty#97] -Results [3]: [i_category#16, sum#98, isEmpty#99] +Functions [1]: [partial_sum(sumsales#24)] -(55) Exchange -Input [3]: [i_category#16, sum#98, isEmpty#99] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(56) CometColumnarExchange +Input [3]: [i_category#16, sum#49, isEmpty#50] +Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(56) HashAggregate [codegen id : 47] -Input [3]: [i_category#16, sum#98, isEmpty#99] +(57) CometHashAggregate +Input [3]: [i_category#16, sum#49, isEmpty#50] Keys [1]: [i_category#16] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#100] -Results [9]: [i_category#16, null AS i_class#101, null AS i_brand#102, null AS i_product_name#103, null AS d_year#104, null AS d_qoy#105, null AS d_moy#106, null AS s_store_id#107, sum(sumsales#26)#100 AS sumsales#108] +Functions [1]: [sum(sumsales#24)] -(57) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110] +(58) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#51, isEmpty#52] -(58) HashAggregate [codegen id : 52] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110] +(59) CometHashAggregate +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#51, isEmpty#52] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(59) HashAggregate [codegen id : 52] -Input [1]: [sumsales#26] +(60) CometHashAggregate +Input [1]: [sumsales#24] Keys: [] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#111, isEmpty#112] -Results [2]: [sum#113, isEmpty#114] +Functions [1]: [partial_sum(sumsales#24)] -(60) Exchange -Input [2]: [sum#113, isEmpty#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] +(61) CometColumnarExchange +Input [2]: [sum#53, isEmpty#54] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(61) HashAggregate [codegen id : 53] -Input [2]: [sum#113, isEmpty#114] +(62) CometHashAggregate +Input [2]: [sum#53, isEmpty#54] Keys: [] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#115] -Results [9]: [null AS i_category#116, null AS i_class#117, null AS i_brand#118, null AS i_product_name#119, null AS d_year#120, null AS d_qoy#121, null AS d_moy#122, null AS s_store_id#123, sum(sumsales#26)#115 AS sumsales#124] - -(62) Union - -(63) Exchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(64) Sort [codegen id : 54] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 - -(65) Window -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#125], [i_category#16], [sumsales#23 DESC NULLS LAST] - -(66) Filter [codegen id : 55] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] -Condition : (rk#125 <= 100) - -(67) TakeOrderedAndProject -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] -Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#125 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] +Functions [1]: [sum(sumsales#24)] + +(63) CometUnion +Child 0 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55] +Child 1 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#56, sumsales#57] +Child 2 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#58, s_store_id#59, sumsales#60] +Child 3 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#61, d_moy#62, s_store_id#63, sumsales#64] +Child 4 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#65, d_qoy#66, d_moy#67, s_store_id#68, sumsales#69] +Child 5 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#70, d_year#71, d_qoy#72, d_moy#73, s_store_id#74, sumsales#75] +Child 6 Input [9]: [i_category#16, i_class#15, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sumsales#82] +Child 7 Input [9]: [i_category#16, i_class#83, i_brand#84, i_product_name#85, d_year#86, d_qoy#87, d_moy#88, s_store_id#89, sumsales#90] +Child 8 Input [9]: [i_category#91, i_class#92, i_brand#93, i_product_name#94, d_year#95, d_qoy#96, d_moy#97, s_store_id#98, sumsales#99] + +(64) CometColumnarExchange +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55] +Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(65) CometSort +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55] +Arguments: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55], [i_category#16 ASC NULLS FIRST, sumsales#55 DESC NULLS LAST] + +(66) ColumnarToRow [codegen id : 37] +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55] + +(67) Window +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55] +Arguments: [rank(sumsales#55) windowspecdefinition(i_category#16, sumsales#55 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#100], [i_category#16], [sumsales#55 DESC NULLS LAST] + +(68) Filter [codegen id : 38] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55, rk#100] +Condition : (rk#100 <= 100) + +(69) TakeOrderedAndProject +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55, rk#100] +Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#55 ASC NULLS FIRST, rk#100 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55, rk#100] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (74) ++- * ColumnarToRow (73) + +- CometProject (72) + +- CometFilter (71) + +- CometScan parquet spark_catalog.default.date_dim (70) (unknown) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#101, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#126) AND (d_month_seq#126 >= 1212)) AND (d_month_seq#126 <= 1223)) AND isnotnull(d_date_sk#7)) +(71) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#101, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#101) AND (d_month_seq#101 >= 1212)) AND (d_month_seq#101 <= 1223)) AND isnotnull(d_date_sk#7)) -(70) CometProject -Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10] +(72) CometProject +Input [5]: [d_date_sk#7, d_month_seq#101, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(71) ColumnarToRow [codegen id : 1] +(73) ColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(72) BroadcastExchange +(74) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index cfac29f8a3..1949bc1306 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -1,17 +1,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (55) + WholeStageCodegen (38) Filter [rk] InputAdapter Window [sumsales,i_category] - WholeStageCodegen (54) - Sort [i_category,sumsales] + WholeStageCodegen (37) + ColumnarToRow InputAdapter - Exchange [i_category] #1 - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + CometSort [i_category,sumsales] + CometColumnarExchange [i_category] #1 + CometUnion + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + RowToColumnar WholeStageCodegen (4) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] @@ -48,75 +48,43 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] - InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - WholeStageCodegen (10) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] - InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - WholeStageCodegen (16) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - WholeStageCodegen (22) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) - HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name] #9 - WholeStageCodegen (28) - HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) - HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - InputAdapter - Exchange [i_category,i_class,i_brand] #10 - WholeStageCodegen (34) - HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) - HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - InputAdapter - Exchange [i_category,i_class] #11 - WholeStageCodegen (40) - HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) - HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - InputAdapter - Exchange [i_category] #12 - WholeStageCodegen (46) - HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - InputAdapter - Exchange #13 - WholeStageCodegen (52) - HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] + CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 + CometHashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,i_brand,sum,isEmpty] + CometColumnarExchange [i_category,i_class,i_brand] #10 + CometHashAggregate [i_category,i_class,i_brand,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,i_class,sum,isEmpty] + CometColumnarExchange [i_category,i_class] #11 + CometHashAggregate [i_category,i_class,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [i_category,sum,isEmpty] + CometColumnarExchange [i_category] #12 + CometHashAggregate [i_category,sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] + CometColumnarExchange #13 + CometHashAggregate [sumsales] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 77dcb698fc..c3d9525d52 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -1,59 +1,63 @@ == Physical Plan == -TakeOrderedAndProject (55) -+- * Project (54) - +- Window (53) - +- * Sort (52) - +- Exchange (51) - +- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- Union (47) - :- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (31) - : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : :- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (29) - : +- * Project (28) - : +- * Filter (27) - : +- Window (26) - : +- * Sort (25) - : +- * HashAggregate (24) - : +- Exchange (23) - : +- * HashAggregate (22) - : +- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * ColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.store_sales (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- ReusedExchange (19) - :- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * HashAggregate (38) - : +- ReusedExchange (37) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- ReusedExchange (42) +TakeOrderedAndProject (59) ++- * Project (58) + +- Window (57) + +- * ColumnarToRow (56) + +- CometSort (55) + +- CometColumnarExchange (54) + +- CometHashAggregate (53) + +- CometColumnarExchange (52) + +- CometHashAggregate (51) + +- CometUnion (50) + :- CometHashAggregate (39) + : +- CometColumnarExchange (38) + : +- RowToColumnar (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (33) + : +- * BroadcastHashJoin LeftSemi BuildRight (32) + : :- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- Window (28) + : +- * ColumnarToRow (27) + : +- CometSort (26) + : +- CometHashAggregate (25) + : +- CometColumnarExchange (24) + : +- RowToColumnar (23) + : +- * HashAggregate (22) + : +- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * ColumnarToRow (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.store_sales (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- ReusedExchange (19) + :- CometHashAggregate (44) + : +- CometColumnarExchange (43) + : +- CometHashAggregate (42) + : +- CometHashAggregate (41) + : +- ReusedExchange (40) + +- CometHashAggregate (49) + +- CometColumnarExchange (48) + +- CometHashAggregate (47) + +- CometHashAggregate (46) + +- ReusedExchange (45) (unknown) Scan parquet spark_catalog.default.store_sales @@ -71,7 +75,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 8] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 60] +(4) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 8] @@ -141,7 +145,7 @@ Join condition: None Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] -(19) ReusedExchange [Reuses operator id: 60] +(19) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#15] (20) BroadcastHashJoin [codegen id : 4] @@ -161,200 +165,195 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum#16] Results [2]: [s_state#14, sum#17] -(23) Exchange +(23) RowToColumnar Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(24) HashAggregate [codegen id : 5] +(24) CometColumnarExchange +Input [2]: [s_state#14, sum#17] +Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(25) CometHashAggregate Input [2]: [s_state#14, sum#17] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] -Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] -(25) Sort [codegen id : 5] -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 +(26) CometSort +Input [3]: [s_state#14, _w0#18, s_state#14] +Arguments: [s_state#14, _w0#18, s_state#14], [s_state#14 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] + +(27) ColumnarToRow [codegen id : 5] +Input [3]: [s_state#14, _w0#18, s_state#14] -(26) Window -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] +(28) Window +Input [3]: [s_state#14, _w0#18, s_state#14] +Arguments: [rank(_w0#18) windowspecdefinition(s_state#14, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#14], [_w0#18 DESC NULLS LAST] -(27) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] -Condition : (ranking#20 <= 5) +(29) Filter [codegen id : 6] +Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] +Condition : (ranking#19 <= 5) -(28) Project [codegen id : 6] +(30) Project [codegen id : 6] Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] +Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] -(29) BroadcastExchange +(31) BroadcastExchange Input [1]: [s_state#14] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 7] Left keys [1]: [s_state#8] Right keys [1]: [s_state#14] Join type: LeftSemi Join condition: None -(31) BroadcastExchange +(33) BroadcastExchange Input [3]: [s_store_sk#6, s_county#7, s_state#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(35) Project [codegen id : 8] Output [3]: [ss_net_profit#2, s_county#7, s_state#8] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] -(34) HashAggregate [codegen id : 8] +(36) HashAggregate [codegen id : 8] Input [3]: [ss_net_profit#2, s_county#7, s_state#8] Keys [2]: [s_state#8, s_county#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [s_state#8, s_county#7, sum#22] +Aggregate Attributes [1]: [sum#20] +Results [3]: [s_state#8, s_county#7, sum#21] -(35) Exchange -Input [3]: [s_state#8, s_county#7, sum#22] -Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(37) RowToColumnar +Input [3]: [s_state#8, s_county#7, sum#21] -(36) HashAggregate [codegen id : 9] -Input [3]: [s_state#8, s_county#7, sum#22] +(38) CometColumnarExchange +Input [3]: [s_state#8, s_county#7, sum#21] +Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(39) CometHashAggregate +Input [3]: [s_state#8, s_county#7, sum#21] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) as decimal(27,2)) AS total_sum#24, s_state#8, s_county#7, 0 AS g_state#25, 0 AS g_county#26, 0 AS lochierarchy#27] -(37) ReusedExchange [Reuses operator id: 35] -Output [3]: [s_state#8, s_county#7, sum#28] +(40) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_state#8, s_county#7, sum#22] -(38) HashAggregate [codegen id : 18] -Input [3]: [s_state#8, s_county#7, sum#28] +(41) CometHashAggregate +Input [3]: [s_state#8, s_county#7, sum#22] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS total_sum#29, s_state#8] -(39) HashAggregate [codegen id : 18] -Input [2]: [total_sum#29, s_state#8] +(42) CometHashAggregate +Input [2]: [total_sum#23, s_state#8] Keys [1]: [s_state#8] -Functions [1]: [partial_sum(total_sum#29)] -Aggregate Attributes [2]: [sum#30, isEmpty#31] -Results [3]: [s_state#8, sum#32, isEmpty#33] +Functions [1]: [partial_sum(total_sum#23)] -(40) Exchange -Input [3]: [s_state#8, sum#32, isEmpty#33] -Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(43) CometColumnarExchange +Input [3]: [s_state#8, sum#24, isEmpty#25] +Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(41) HashAggregate [codegen id : 19] -Input [3]: [s_state#8, sum#32, isEmpty#33] +(44) CometHashAggregate +Input [3]: [s_state#8, sum#24, isEmpty#25] Keys [1]: [s_state#8] -Functions [1]: [sum(total_sum#29)] -Aggregate Attributes [1]: [sum(total_sum#29)#34] -Results [6]: [sum(total_sum#29)#34 AS total_sum#35, s_state#8, null AS s_county#36, 0 AS g_state#37, 1 AS g_county#38, 1 AS lochierarchy#39] +Functions [1]: [sum(total_sum#23)] -(42) ReusedExchange [Reuses operator id: 35] -Output [3]: [s_state#8, s_county#7, sum#40] +(45) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_state#8, s_county#7, sum#26] -(43) HashAggregate [codegen id : 28] -Input [3]: [s_state#8, s_county#7, sum#40] +(46) CometHashAggregate +Input [3]: [s_state#8, s_county#7, sum#26] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS total_sum#29] -(44) HashAggregate [codegen id : 28] -Input [1]: [total_sum#29] +(47) CometHashAggregate +Input [1]: [total_sum#23] Keys: [] -Functions [1]: [partial_sum(total_sum#29)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [2]: [sum#43, isEmpty#44] +Functions [1]: [partial_sum(total_sum#23)] -(45) Exchange -Input [2]: [sum#43, isEmpty#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(48) CometColumnarExchange +Input [2]: [sum#27, isEmpty#28] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(46) HashAggregate [codegen id : 29] -Input [2]: [sum#43, isEmpty#44] +(49) CometHashAggregate +Input [2]: [sum#27, isEmpty#28] Keys: [] -Functions [1]: [sum(total_sum#29)] -Aggregate Attributes [1]: [sum(total_sum#29)#45] -Results [6]: [sum(total_sum#29)#45 AS total_sum#46, null AS s_state#47, null AS s_county#48, 1 AS g_state#49, 1 AS g_county#50, 2 AS lochierarchy#51] +Functions [1]: [sum(total_sum#23)] -(47) Union +(50) CometUnion +Child 0 Input [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] +Child 1 Input [6]: [total_sum#33, s_state#8, s_county#34, g_state#35, g_county#36, lochierarchy#37] +Child 2 Input [6]: [total_sum#38, s_state#39, s_county#40, g_state#41, g_county#42, lochierarchy#43] -(48) HashAggregate [codegen id : 30] -Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +(51) CometHashAggregate +Input [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] +Keys [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -(49) Exchange -Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -Arguments: hashpartitioning(total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(52) CometColumnarExchange +Input [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] +Arguments: hashpartitioning(total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(50) HashAggregate [codegen id : 31] -Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +(53) CometHashAggregate +Input [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] +Keys [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] Functions: [] -Aggregate Attributes: [] -Results [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, CASE WHEN (g_county#26 = 0) THEN s_state#8 END AS _w0#52] -(51) Exchange -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] -Arguments: hashpartitioning(lochierarchy#27, _w0#52, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(54) CometColumnarExchange +Input [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44] +Arguments: hashpartitioning(lochierarchy#32, _w0#44, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(55) CometSort +Input [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44] +Arguments: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44], [lochierarchy#32 ASC NULLS FIRST, _w0#44 ASC NULLS FIRST, total_sum#29 DESC NULLS LAST] -(52) Sort [codegen id : 32] -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#52 ASC NULLS FIRST, total_sum#24 DESC NULLS LAST], false, 0 +(56) ColumnarToRow [codegen id : 25] +Input [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44] -(53) Window -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] -Arguments: [rank(total_sum#24) windowspecdefinition(lochierarchy#27, _w0#52, total_sum#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#53], [lochierarchy#27, _w0#52], [total_sum#24 DESC NULLS LAST] +(57) Window +Input [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44] +Arguments: [rank(total_sum#29) windowspecdefinition(lochierarchy#32, _w0#44, total_sum#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#45], [lochierarchy#32, _w0#44], [total_sum#29 DESC NULLS LAST] -(54) Project [codegen id : 33] -Output [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] -Input [6]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52, rank_within_parent#53] +(58) Project [codegen id : 26] +Output [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, rank_within_parent#45] +Input [6]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44, rank_within_parent#45] -(55) TakeOrderedAndProject -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#53 ASC NULLS FIRST], [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] +(59) TakeOrderedAndProject +Input [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, rank_within_parent#45] +Arguments: 100, [lochierarchy#32 DESC NULLS LAST, CASE WHEN (lochierarchy#32 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#45 ASC NULLS FIRST], [total_sum#29, s_state#8, s_county#7, lochierarchy#32, rank_within_parent#45] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (60) -+- * ColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.date_dim (56) +BroadcastExchange (64) ++- * ColumnarToRow (63) + +- CometProject (62) + +- CometFilter (61) + +- CometScan parquet spark_catalog.default.date_dim (60) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#54] +Output [2]: [d_date_sk#5, d_month_seq#46] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#54] -Condition : (((isnotnull(d_month_seq#54) AND (d_month_seq#54 >= 1212)) AND (d_month_seq#54 <= 1223)) AND isnotnull(d_date_sk#5)) +(61) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#46] +Condition : (((isnotnull(d_month_seq#46) AND (d_month_seq#46 >= 1212)) AND (d_month_seq#46 <= 1223)) AND isnotnull(d_date_sk#5)) -(58) CometProject -Input [2]: [d_date_sk#5, d_month_seq#54] +(62) CometProject +Input [2]: [d_date_sk#5, d_month_seq#46] Arguments: [d_date_sk#5], [d_date_sk#5] -(59) ColumnarToRow [codegen id : 1] +(63) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(60) BroadcastExchange +(64) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 663f828f46..4719e8ea07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -1,99 +1,89 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (33) + WholeStageCodegen (26) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (32) - Sort [lochierarchy,_w0,total_sum] + WholeStageCodegen (25) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (31) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] [_w0] - InputAdapter - Exchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (30) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (9) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] - InputAdapter - Exchange [s_state,s_county] #3 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometSort [lochierarchy,_w0,total_sum] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + CometHashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + CometUnion + CometHashAggregate [s_state,s_county,sum] + CometColumnarExchange [s_state,s_county] #3 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + BroadcastHashJoin [s_state,s_state] + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - BroadcastHashJoin [s_state,s_state] - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (6) + Project [s_state] + Filter [ranking] InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] + Window [_w0,s_state] + WholeStageCodegen (5) + ColumnarToRow InputAdapter - Window [_w0,s_state] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometSort [s_state,_w0] + CometHashAggregate [s_state,sum] + CometColumnarExchange [s_state] #7 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - InputAdapter - Exchange [s_state] #9 - WholeStageCodegen (18) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - InputAdapter - Exchange #10 - WholeStageCodegen (28) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #4 + CometHashAggregate [s_state,sum,isEmpty] + CometColumnarExchange [s_state] #9 + CometHashAggregate [s_state,total_sum] + CometHashAggregate [s_state,s_county,sum] + ReusedExchange [s_state,s_county,sum] #3 + CometHashAggregate [sum,isEmpty] + CometColumnarExchange #10 + CometHashAggregate [total_sum] + CometHashAggregate [s_state,s_county,sum] + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index 46b9e51d5e..9fe23081f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -1,74 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * SortMergeJoin LeftOuter (65) - :- * Sort (58) - : +- Exchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * ColumnarToRow (3) - : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * ColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * ColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * ColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * ColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * ColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * ColumnarToRow (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.promotion (51) - +- * ColumnarToRow (64) - +- CometSort (63) - +- CometExchange (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.catalog_returns (59) +* ColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometHashAggregate (69) + +- CometColumnarExchange (68) + +- CometHashAggregate (67) + +- CometProject (66) + +- CometSortMergeJoin (65) + :- CometSort (59) + : +- CometColumnarExchange (58) + : +- RowToColumnar (57) + : +- * Project (56) + : +- * BroadcastHashJoin LeftOuter BuildRight (55) + : :- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Project (15) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : : :- * Project (9) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : : : : :- * ColumnarToRow (3) + : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- BroadcastExchange (7) + : : : : : : : : : +- * ColumnarToRow (6) + : : : : : : : : : +- CometFilter (5) + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) + : : : : : : : : +- BroadcastExchange (13) + : : : : : : : : +- * ColumnarToRow (12) + : : : : : : : : +- CometFilter (11) + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * ColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- CometProject (24) + : : : : : : +- CometFilter (23) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) + : : : : : +- BroadcastExchange (33) + : : : : : +- * ColumnarToRow (32) + : : : : : +- CometProject (31) + : : : : : +- CometFilter (30) + : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) + : : : : +- ReusedExchange (36) + : : : +- BroadcastExchange (42) + : : : +- * ColumnarToRow (41) + : : : +- CometFilter (40) + : : : +- CometScan parquet spark_catalog.default.date_dim (39) + : : +- BroadcastExchange (48) + : : +- * ColumnarToRow (47) + : : +- CometFilter (46) + : : +- CometScan parquet spark_catalog.default.date_dim (45) + : +- BroadcastExchange (54) + : +- * ColumnarToRow (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.promotion (51) + +- CometSort (64) + +- CometColumnarExchange (63) + +- CometProject (62) + +- CometFilter (61) + +- CometScan parquet spark_catalog.default.catalog_returns (60) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -235,7 +236,7 @@ Join condition: None Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -(36) ReusedExchange [Reuses operator id: 75] +(36) ReusedExchange [Reuses operator id: 76] Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (37) BroadcastHashJoin [codegen id : 10] @@ -332,13 +333,16 @@ Join condition: None Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] -(57) Exchange +(57) RowToColumnar Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(58) Sort [codegen id : 11] +(58) CometColumnarExchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(59) CometSort +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.catalog_returns Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] @@ -347,86 +351,81 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) CometFilter +(61) CometFilter Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) -(61) CometProject +(62) CometProject Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] -(62) CometExchange +(63) CometColumnarExchange Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(63) CometSort +(64) CometSort Input [2]: [cr_item_sk#30, cr_order_number#31] Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] -(64) ColumnarToRow [codegen id : 12] -Input [2]: [cr_item_sk#30, cr_order_number#31] +(65) CometSortMergeJoin +Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Right output [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter -(65) SortMergeJoin [codegen id : 13] -Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#30, cr_order_number#31] -Join type: LeftOuter -Join condition: None - -(66) Project [codegen id : 13] -Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(66) CometProject Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] +Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(67) HashAggregate [codegen id : 13] +(67) CometHashAggregate Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -(68) Exchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(68) CometColumnarExchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(69) HashAggregate [codegen id : 14] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +(69) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#35 AS no_promo#36, count(1)#35 AS promo#37, count(1)#35 AS total_cnt#38] -(70) TakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] -Arguments: 100, [total_cnt#38 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] +(70) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), 100, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] + +(71) ColumnarToRow [codegen id : 11] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] -Condition : ((((isnotnull(d_year#39) AND (d_year#39 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(73) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +(74) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(75) BroadcastExchange +(76) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index 5eb8ea5275..5bf7ab343a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -1,114 +1,108 @@ -TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - WholeStageCodegen (14) - HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] - InputAdapter - Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - WholeStageCodegen (13) - HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] - Project [w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - InputAdapter - WholeStageCodegen (11) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - ColumnarToRow - InputAdapter - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] +WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] + CometColumnarExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq] + CometProject [w_warehouse_name,i_item_desc,d_week_seq] + CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + RowToColumnar + WholeStageCodegen (10) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + ColumnarToRow + InputAdapter + CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) + BroadcastExchange #6 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometProject [cd_demo_sk] + CometFilter [cd_marital_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + BroadcastExchange #8 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [d_week_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) + BroadcastExchange #9 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometFilter [d_week_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) + BroadcastExchange #10 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometColumnarExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index 776fad0078..46315a3c4f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -1,75 +1,83 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * ColumnarToRow (36) - : : : +- CometFilter (35) - : : : +- CometScan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) - +- BroadcastExchange (68) - +- * HashAggregate (67) - +- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * ColumnarToRow (55) - : : +- CometFilter (54) - : : +- CometScan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) +TakeOrderedAndProject (79) ++- * Project (78) + +- * BroadcastHashJoin Inner BuildRight (77) + :- * Project (58) + : +- * BroadcastHashJoin Inner BuildRight (57) + : :- * BroadcastHashJoin Inner BuildRight (37) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometColumnarExchange (15) + : : : +- RowToColumnar (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (36) + : : +- * ColumnarToRow (35) + : : +- CometHashAggregate (34) + : : +- CometColumnarExchange (33) + : : +- RowToColumnar (32) + : : +- * HashAggregate (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * Project (27) + : : : +- * BroadcastHashJoin Inner BuildRight (26) + : : : :- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.store_sales (22) + : : +- ReusedExchange (28) + : +- BroadcastExchange (56) + : +- * ColumnarToRow (55) + : +- CometFilter (54) + : +- CometHashAggregate (53) + : +- CometColumnarExchange (52) + : +- RowToColumnar (51) + : +- * HashAggregate (50) + : +- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (46) + : : +- * BroadcastHashJoin Inner BuildRight (45) + : : :- * ColumnarToRow (40) + : : : +- CometFilter (39) + : : : +- CometScan parquet spark_catalog.default.customer (38) + : : +- BroadcastExchange (44) + : : +- * ColumnarToRow (43) + : : +- CometFilter (42) + : : +- CometScan parquet spark_catalog.default.web_sales (41) + : +- ReusedExchange (47) + +- BroadcastExchange (76) + +- * ColumnarToRow (75) + +- CometHashAggregate (74) + +- CometColumnarExchange (73) + +- RowToColumnar (72) + +- * HashAggregate (71) + +- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * ColumnarToRow (61) + : : +- CometFilter (60) + : : +- CometScan parquet spark_catalog.default.customer (59) + : +- BroadcastExchange (65) + : +- * ColumnarToRow (64) + : +- CometFilter (63) + : +- CometScan parquet spark_catalog.default.web_sales (62) + +- ReusedExchange (68) (unknown) Scan parquet spark_catalog.default.customer @@ -115,7 +123,7 @@ Join condition: None Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 75] +(10) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#9, d_year#10] (11) BroadcastHashJoin [codegen id : 3] @@ -135,296 +143,312 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] Aggregate Attributes [1]: [sum#11] Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -(14) Exchange +(14) RowToColumnar Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 16] +(15) CometColumnarExchange +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] -Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] -(16) Filter [codegen id : 16] -Input [2]: [customer_id#14, year_total#15] -Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) +(17) CometFilter +Input [2]: [customer_id#13, year_total#14] +Condition : (isnotnull(year_total#14) AND (year_total#14 > 0.00)) + +(18) ColumnarToRow [codegen id : 16] +Input [2]: [customer_id#13, year_total#14] (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +Output [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter -Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] -Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) +(20) CometFilter +Input [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_customer_id#16)) -(19) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +(21) ColumnarToRow [codegen id : 6] +Input [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#20) +(23) CometFilter +Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Condition : isnotnull(ss_customer_sk#19) -(22) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +(24) ColumnarToRow [codegen id : 4] +Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -(23) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +(25) BroadcastExchange +Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#16] -Right keys [1]: [ss_customer_sk#20] +(26) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#15] +Right keys [1]: [ss_customer_sk#19] Join type: Inner Join condition: None -(25) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] -Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +(27) Project [codegen id : 6] +Output [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21] +Input [7]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18, ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -(26) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#24, d_year#25] +(28) ReusedExchange [Reuses operator id: 87] +Output [2]: [d_date_sk#23, d_year#24] -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#24] +(29) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#21] +Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] - -(29) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] -Aggregate Attributes [1]: [sum#26] -Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] - -(30) Exchange -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] -Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] -Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] - -(32) BroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +(30) Project [codegen id : 6] +Output [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] +Input [7]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21, d_date_sk#23, d_year#24] + +(31) HashAggregate [codegen id : 6] +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] +Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#20))] +Aggregate Attributes [1]: [sum#25] +Results [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] + +(32) RowToColumnar +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] + +(33) CometColumnarExchange +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] +Arguments: hashpartitioning(c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometHashAggregate +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] +Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] +Functions [1]: [sum(UnscaledValue(ss_net_paid#20))] + +(35) ColumnarToRow [codegen id : 7] +Input [4]: [customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30] + +(36) BroadcastExchange +Input [4]: [customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#28] +(37) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#27] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Output [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) +(39) CometFilter +Input [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] +Condition : (isnotnull(c_customer_sk#31) AND isnotnull(c_customer_id#32)) -(36) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +(40) ColumnarToRow [codegen id : 10] +Input [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Output [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_bill_customer_sk#36) +(42) CometFilter +Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] +Condition : isnotnull(ws_bill_customer_sk#35) -(39) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(43) ColumnarToRow [codegen id : 8] +Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] -(40) BroadcastExchange -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(44) BroadcastExchange +Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#32] -Right keys [1]: [ws_bill_customer_sk#36] +(45) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#31] +Right keys [1]: [ws_bill_customer_sk#35] Join type: Inner Join condition: None -(42) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] -Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(46) Project [codegen id : 10] +Output [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, ws_sold_date_sk#37] +Input [7]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] -(43) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#40, d_year#41] +(47) ReusedExchange [Reuses operator id: 83] +Output [2]: [d_date_sk#39, d_year#40] -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#37] +Right keys [1]: [d_date_sk#39] Join type: Inner Join condition: None -(45) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] - -(46) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] -Aggregate Attributes [1]: [sum#42] -Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] - -(47) Exchange -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(48) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] -Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] -Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] - -(49) Filter [codegen id : 11] -Input [2]: [customer_id#45, year_total#46] -Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) - -(50) BroadcastExchange -Input [2]: [customer_id#45, year_total#46] +(49) Project [codegen id : 10] +Output [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, d_year#40] +Input [7]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, ws_sold_date_sk#37, d_date_sk#39, d_year#40] + +(50) HashAggregate [codegen id : 10] +Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, d_year#40] +Keys [4]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#36))] +Aggregate Attributes [1]: [sum#41] +Results [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] + +(51) RowToColumnar +Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] + +(52) CometColumnarExchange +Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] +Arguments: hashpartitioning(c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(53) CometHashAggregate +Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] +Keys [4]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40] +Functions [1]: [sum(UnscaledValue(ws_net_paid#36))] + +(54) CometFilter +Input [2]: [customer_id#43, year_total#44] +Condition : (isnotnull(year_total#44) AND (year_total#44 > 0.00)) + +(55) ColumnarToRow [codegen id : 11] +Input [2]: [customer_id#43, year_total#44] + +(56) BroadcastExchange +Input [2]: [customer_id#43, year_total#44] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#45] +(57) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#43] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] -Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] +(58) Project [codegen id : 16] +Output [7]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, year_total#44] +Input [8]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, customer_id#43, year_total#44] (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +Output [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter -Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] -Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) +(60) CometFilter +Input [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] +Condition : (isnotnull(c_customer_sk#45) AND isnotnull(c_customer_id#46)) -(55) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +(61) ColumnarToRow [codegen id : 14] +Input [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Output [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#52)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_bill_customer_sk#51) +(63) CometFilter +Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_customer_sk#49) -(58) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +(64) ColumnarToRow [codegen id : 12] +Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] -(59) BroadcastExchange -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +(65) BroadcastExchange +Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#47] -Right keys [1]: [ws_bill_customer_sk#51] +(66) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#45] +Right keys [1]: [ws_bill_customer_sk#49] Join type: Inner Join condition: None -(61) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] -Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +(67) Project [codegen id : 14] +Output [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, ws_sold_date_sk#51] +Input [7]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48, ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] -(62) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#55, d_year#56] +(68) ReusedExchange [Reuses operator id: 87] +Output [2]: [d_date_sk#53, d_year#54] -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#53] -Right keys [1]: [d_date_sk#55] +(69) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#53] Join type: Inner Join condition: None -(64) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] - -(65) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] -Aggregate Attributes [1]: [sum#57] -Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] - -(66) Exchange -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(67) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] -Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#52))#44] -Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#52))#44,17,2) AS year_total#60] - -(68) BroadcastExchange -Input [2]: [customer_id#59, year_total#60] +(70) Project [codegen id : 14] +Output [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, d_year#54] +Input [7]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, ws_sold_date_sk#51, d_date_sk#53, d_year#54] + +(71) HashAggregate [codegen id : 14] +Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, d_year#54] +Keys [4]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#50))] +Aggregate Attributes [1]: [sum#55] +Results [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] + +(72) RowToColumnar +Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] + +(73) CometColumnarExchange +Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] +Arguments: hashpartitioning(c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(74) CometHashAggregate +Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] +Keys [4]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54] +Functions [1]: [sum(UnscaledValue(ws_net_paid#50))] + +(75) ColumnarToRow [codegen id : 15] +Input [2]: [customer_id#57, year_total#58] + +(76) BroadcastExchange +Input [2]: [customer_id#57, year_total#58] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(69) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#59] +(77) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#57] Join type: Inner -Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) +Join condition: (CASE WHEN (year_total#44 > 0.00) THEN (year_total#58 / year_total#44) END > CASE WHEN (year_total#14 > 0.00) THEN (year_total#30 / year_total#14) END) -(70) Project [codegen id : 16] -Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] +(78) Project [codegen id : 16] +Output [3]: [customer_id#27, customer_first_name#28, customer_last_name#29] +Input [9]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, year_total#44, customer_id#57, year_total#58] -(71) TakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +(79) TakeOrderedAndProject +Input [3]: [customer_id#27, customer_first_name#28, customer_last_name#29] +Arguments: 100, [customer_first_name#28 ASC NULLS FIRST, customer_id#27 ASC NULLS FIRST, customer_last_name#29 ASC NULLS FIRST], [customer_id#27, customer_first_name#28, customer_last_name#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (83) ++- * ColumnarToRow (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) (unknown) Scan parquet spark_catalog.default.date_dim @@ -434,44 +458,44 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(81) CometFilter Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(74) ColumnarToRow [codegen id : 1] +(82) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(75) BroadcastExchange +(83) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 +BroadcastExchange (87) ++- * ColumnarToRow (86) + +- CometFilter (85) + +- CometScan parquet spark_catalog.default.date_dim (84) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +Output [2]: [d_date_sk#23, d_year#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) +(85) CometFilter +Input [2]: [d_date_sk#23, d_year#24] +Condition : (((isnotnull(d_year#24) AND (d_year#24 = 2002)) AND d_year#24 IN (2001,2002)) AND isnotnull(d_date_sk#23)) -(78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#25] +(86) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#23, d_year#24] -(79) BroadcastExchange -Input [2]: [d_date_sk#24, d_year#25] +(87) BroadcastExchange +Input [2]: [d_date_sk#23, d_year#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#22 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index 26989b0c00..e8b5c93db2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -5,43 +5,13 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [year_total] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + RowToColumnar + WholeStageCodegen (3) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -52,71 +22,109 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + ColumnarToRow + InputAdapter + CometFilter [year_total] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + ColumnarToRow InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 3922f7efe6..98da31a346 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -1,136 +1,138 @@ == Physical Plan == -TakeOrderedAndProject (132) -+- * Project (131) - +- * SortMergeJoin Inner (130) - :- * Sort (71) - : +- Exchange (70) - : +- * Filter (69) - : +- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- Union (62) - : :- * Project (23) - : : +- * SortMergeJoin LeftOuter (22) - : : :- * Sort (15) - : : : +- Exchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- * ColumnarToRow (21) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : :- * Project (42) - : : +- * SortMergeJoin LeftOuter (41) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : :- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * ColumnarToRow (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) - : : : : +- ReusedExchange (27) - : : : +- ReusedExchange (30) - : : +- * ColumnarToRow (40) - : : +- CometSort (39) - : : +- CometExchange (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometScan parquet spark_catalog.default.store_returns (35) - : +- * Project (61) - : +- * SortMergeJoin LeftOuter (60) - : :- * Sort (53) - : : +- Exchange (52) - : : +- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * ColumnarToRow (45) - : : : : +- CometFilter (44) - : : : : +- CometScan parquet spark_catalog.default.web_sales (43) - : : : +- ReusedExchange (46) - : : +- ReusedExchange (49) - : +- * ColumnarToRow (59) - : +- CometSort (58) - : +- CometExchange (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan parquet spark_catalog.default.web_returns (54) - +- * Sort (129) - +- Exchange (128) - +- * Filter (127) - +- * HashAggregate (126) - +- Exchange (125) - +- * HashAggregate (124) - +- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- Union (120) - :- * Project (87) - : +- * SortMergeJoin LeftOuter (86) - : :- * Sort (82) - : : +- Exchange (81) - : : +- * Project (80) - : : +- * BroadcastHashJoin Inner BuildRight (79) - : : :- * Project (77) - : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : :- * ColumnarToRow (74) - : : : : +- CometFilter (73) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) - : : : +- ReusedExchange (75) - : : +- ReusedExchange (78) - : +- * ColumnarToRow (85) - : +- CometSort (84) - : +- ReusedExchange (83) - :- * Project (103) - : +- * SortMergeJoin LeftOuter (102) - : :- * Sort (98) - : : +- Exchange (97) - : : +- * Project (96) - : : +- * BroadcastHashJoin Inner BuildRight (95) - : : :- * Project (93) - : : : +- * BroadcastHashJoin Inner BuildRight (92) - : : : :- * ColumnarToRow (90) - : : : : +- CometFilter (89) - : : : : +- CometScan parquet spark_catalog.default.store_sales (88) - : : : +- ReusedExchange (91) - : : +- ReusedExchange (94) - : +- * ColumnarToRow (101) - : +- CometSort (100) - : +- ReusedExchange (99) - +- * Project (119) - +- * SortMergeJoin LeftOuter (118) - :- * Sort (114) - : +- Exchange (113) - : +- * Project (112) - : +- * BroadcastHashJoin Inner BuildRight (111) - : :- * Project (109) - : : +- * BroadcastHashJoin Inner BuildRight (108) - : : :- * ColumnarToRow (106) - : : : +- CometFilter (105) - : : : +- CometScan parquet spark_catalog.default.web_sales (104) - : : +- ReusedExchange (107) - : +- ReusedExchange (110) - +- * ColumnarToRow (117) - +- CometSort (116) - +- ReusedExchange (115) +TakeOrderedAndProject (134) ++- * Project (133) + +- * SortMergeJoin Inner (132) + :- * ColumnarToRow (72) + : +- CometSort (71) + : +- CometColumnarExchange (70) + : +- CometFilter (69) + : +- CometHashAggregate (68) + : +- CometColumnarExchange (67) + : +- CometHashAggregate (66) + : +- CometHashAggregate (65) + : +- CometColumnarExchange (64) + : +- CometHashAggregate (63) + : +- CometUnion (62) + : :- CometProject (23) + : : +- CometSortMergeJoin (22) + : : :- CometSort (16) + : : : +- CometColumnarExchange (15) + : : : +- RowToColumnar (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- ReusedExchange (11) + : : +- CometSort (21) + : : +- CometColumnarExchange (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : :- CometProject (42) + : : +- CometSortMergeJoin (41) + : : :- CometSort (35) + : : : +- CometColumnarExchange (34) + : : : +- RowToColumnar (33) + : : : +- * Project (32) + : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : :- * Project (29) + : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : :- * ColumnarToRow (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- CometSort (40) + : : +- CometColumnarExchange (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.store_returns (36) + : +- CometProject (61) + : +- CometSortMergeJoin (60) + : :- CometSort (54) + : : +- CometColumnarExchange (53) + : : +- RowToColumnar (52) + : : +- * Project (51) + : : +- * BroadcastHashJoin Inner BuildRight (50) + : : :- * Project (48) + : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : :- * ColumnarToRow (45) + : : : : +- CometFilter (44) + : : : : +- CometScan parquet spark_catalog.default.web_sales (43) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- CometSort (59) + : +- CometColumnarExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.web_returns (55) + +- * ColumnarToRow (131) + +- CometSort (130) + +- CometColumnarExchange (129) + +- CometFilter (128) + +- CometHashAggregate (127) + +- CometColumnarExchange (126) + +- CometHashAggregate (125) + +- CometHashAggregate (124) + +- CometColumnarExchange (123) + +- CometHashAggregate (122) + +- CometUnion (121) + :- CometProject (88) + : +- CometSortMergeJoin (87) + : :- CometSort (84) + : : +- CometColumnarExchange (83) + : : +- RowToColumnar (82) + : : +- * Project (81) + : : +- * BroadcastHashJoin Inner BuildRight (80) + : : :- * Project (78) + : : : +- * BroadcastHashJoin Inner BuildRight (77) + : : : :- * ColumnarToRow (75) + : : : : +- CometFilter (74) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) + : : : +- ReusedExchange (76) + : : +- ReusedExchange (79) + : +- CometSort (86) + : +- ReusedExchange (85) + :- CometProject (104) + : +- CometSortMergeJoin (103) + : :- CometSort (100) + : : +- CometColumnarExchange (99) + : : +- RowToColumnar (98) + : : +- * Project (97) + : : +- * BroadcastHashJoin Inner BuildRight (96) + : : :- * Project (94) + : : : +- * BroadcastHashJoin Inner BuildRight (93) + : : : :- * ColumnarToRow (91) + : : : : +- CometFilter (90) + : : : : +- CometScan parquet spark_catalog.default.store_sales (89) + : : : +- ReusedExchange (92) + : : +- ReusedExchange (95) + : +- CometSort (102) + : +- ReusedExchange (101) + +- CometProject (120) + +- CometSortMergeJoin (119) + :- CometSort (116) + : +- CometColumnarExchange (115) + : +- RowToColumnar (114) + : +- * Project (113) + : +- * BroadcastHashJoin Inner BuildRight (112) + : :- * Project (110) + : : +- * BroadcastHashJoin Inner BuildRight (109) + : : :- * ColumnarToRow (107) + : : : +- CometFilter (106) + : : : +- CometScan parquet spark_catalog.default.web_sales (105) + : : +- ReusedExchange (108) + : +- ReusedExchange (111) + +- CometSort (118) + +- ReusedExchange (117) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -180,7 +182,7 @@ Join condition: None Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 136] +(11) ReusedExchange [Reuses operator id: 138] Output [2]: [d_date_sk#13, d_year#14] (12) BroadcastHashJoin [codegen id : 3] @@ -193,13 +195,16 @@ Join condition: None Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -(14) Exchange +(14) RowToColumnar Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) Sort [codegen id : 4] +(15) CometColumnarExchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometSort +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] @@ -208,34 +213,30 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(17) CometFilter +(18) CometFilter Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) -(18) CometProject +(19) CometProject Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(19) CometExchange +(20) CometColumnarExchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) CometSort +(21) CometSort Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] -(21) ColumnarToRow [codegen id : 5] -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +(22) CometSortMergeJoin +Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter -(22) SortMergeJoin [codegen id : 6] -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#16, cr_item_sk#15] -Join type: LeftOuter -Join condition: None - -(23) Project [codegen id : 6] -Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] +(23) CometProject Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] (unknown) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] @@ -249,42 +250,45 @@ ReadSchema: struct -(36) CometFilter +(37) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(37) CometProject +(38) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(38) CometExchange +(39) CometColumnarExchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(39) CometSort +(40) CometSort Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(40) ColumnarToRow [codegen id : 11] -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(41) SortMergeJoin [codegen id : 12] -Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] -Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] -Join type: LeftOuter -Join condition: None +(41) CometSortMergeJoin +Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(42) Project [codegen id : 12] -Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] +(42) CometProject Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] (unknown) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] @@ -334,42 +334,45 @@ ReadSchema: struct -(55) CometFilter +(56) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(56) CometProject +(57) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(57) CometExchange +(58) CometColumnarExchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(58) CometSort +(59) CometSort Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(59) ColumnarToRow [codegen id : 17] -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(60) SortMergeJoin [codegen id : 18] -Left keys [2]: [ws_order_number#43, ws_item_sk#42] -Right keys [2]: [wr_order_number#56, wr_item_sk#55] -Join type: LeftOuter -Join condition: None +(60) CometSortMergeJoin +Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(61) Project [codegen id : 18] -Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] +(61) CometProject Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] -(62) Union +(62) CometUnion +Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(63) HashAggregate [codegen id : 19] +(63) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(64) Exchange +(64) CometColumnarExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(65) HashAggregate [codegen id : 20] +(65) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(66) HashAggregate [codegen id : 20] +(66) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum#62, sum#63] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -(67) Exchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(67) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) HashAggregate [codegen id : 21] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] +(68) CometHashAggregate +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] -(69) Filter [codegen id : 21] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] -Condition : isnotnull(sales_cnt#68) +(69) CometFilter +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Condition : isnotnull(sales_cnt#64) -(70) Exchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(70) CometColumnarExchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(71) Sort [codegen id : 22] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] -Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 +(71) CometSort +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] + +(72) ColumnarToRow [codegen id : 10] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(73) CometFilter -Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] -Condition : isnotnull(cs_item_sk#70) +(74) CometFilter +Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Condition : isnotnull(cs_item_sk#66) -(74) ColumnarToRow [codegen id : 25] -Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +(75) ColumnarToRow [codegen id : 13] +Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -(75) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(76) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(76) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [cs_item_sk#70] -Right keys [1]: [i_item_sk#76] +(77) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_item_sk#66] +Right keys [1]: [i_item_sk#72] Join type: Inner Join condition: None -(77) Project [codegen id : 25] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(78) Project [codegen id : 13] +Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(78) ReusedExchange [Reuses operator id: 140] -Output [2]: [d_date_sk#81, d_year#82] +(79) ReusedExchange [Reuses operator id: 142] +Output [2]: [d_date_sk#77, d_year#78] -(79) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [cs_sold_date_sk#74] -Right keys [1]: [d_date_sk#81] +(80) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_sold_date_sk#70] +Right keys [1]: [d_date_sk#77] Join type: Inner Join condition: None -(80) Project [codegen id : 25] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] - -(81) Exchange -Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(81) Project [codegen id : 13] +Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -(82) Sort [codegen id : 26] -Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 +(82) RowToColumnar +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -(83) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +(83) CometColumnarExchange +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] (84) CometSort -Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -Arguments: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86], [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST] +Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] -(85) ColumnarToRow [codegen id : 27] -Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +(85) ReusedExchange [Reuses operator id: 20] +Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -(86) SortMergeJoin [codegen id : 28] -Left keys [2]: [cs_order_number#71, cs_item_sk#70] -Right keys [2]: [cr_order_number#84, cr_item_sk#83] -Join type: LeftOuter -Join condition: None +(86) CometSort +Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] + +(87) CometSortMergeJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter -(87) Project [codegen id : 28] -Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#20, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#21] -Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +(88) CometProject +Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] (unknown) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(89) CometFilter -Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -Condition : isnotnull(ss_item_sk#87) +(90) CometFilter +Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Condition : isnotnull(ss_item_sk#83) -(90) ColumnarToRow [codegen id : 31] -Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +(91) ColumnarToRow [codegen id : 16] +Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -(91) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(92) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(92) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [ss_item_sk#87] -Right keys [1]: [i_item_sk#93] +(93) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ss_item_sk#83] +Right keys [1]: [i_item_sk#89] Join type: Inner Join condition: None -(93) Project [codegen id : 31] -Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(94) Project [codegen id : 16] +Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(94) ReusedExchange [Reuses operator id: 140] -Output [2]: [d_date_sk#98, d_year#99] +(95) ReusedExchange [Reuses operator id: 142] +Output [2]: [d_date_sk#94, d_year#95] -(95) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [ss_sold_date_sk#91] -Right keys [1]: [d_date_sk#98] +(96) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ss_sold_date_sk#87] +Right keys [1]: [d_date_sk#94] Join type: Inner Join condition: None -(96) Project [codegen id : 31] -Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] - -(97) Exchange -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(97) Project [codegen id : 16] +Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -(98) Sort [codegen id : 32] -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 +(98) RowToColumnar +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -(99) ReusedExchange [Reuses operator id: 38] -Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(99) CometColumnarExchange +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] (100) CometSort -Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103], [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST] +Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] -(101) ColumnarToRow [codegen id : 33] -Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(101) ReusedExchange [Reuses operator id: 39] +Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -(102) SortMergeJoin [codegen id : 34] -Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] -Right keys [2]: [sr_ticket_number#101, sr_item_sk#100] -Join type: LeftOuter -Join condition: None +(102) CometSort +Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] + +(103) CometSortMergeJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter -(103) Project [codegen id : 34] -Output [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#40, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#41] -Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(104) CometProject +Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] (unknown) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] +PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(105) CometFilter -Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Condition : isnotnull(ws_item_sk#104) +(106) CometFilter +Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Condition : isnotnull(ws_item_sk#100) -(106) ColumnarToRow [codegen id : 37] -Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +(107) ColumnarToRow [codegen id : 19] +Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -(107) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +(108) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(108) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_item_sk#104] -Right keys [1]: [i_item_sk#110] +(109) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_item_sk#100] +Right keys [1]: [i_item_sk#106] Join type: Inner Join condition: None -(109) Project [codegen id : 37] -Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +(110) Project [codegen id : 19] +Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(110) ReusedExchange [Reuses operator id: 140] -Output [2]: [d_date_sk#115, d_year#116] +(111) ReusedExchange [Reuses operator id: 142] +Output [2]: [d_date_sk#111, d_year#112] -(111) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#108] -Right keys [1]: [d_date_sk#115] +(112) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_sold_date_sk#104] +Right keys [1]: [d_date_sk#111] Join type: Inner Join condition: None -(112) Project [codegen id : 37] -Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] - -(113) Exchange -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, [plan_id=13] +(113) Project [codegen id : 19] +Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -(114) Sort [codegen id : 38] -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST], false, 0 +(114) RowToColumnar +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -(115) ReusedExchange [Reuses operator id: 57] -Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +(115) CometColumnarExchange +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] (116) CometSort -Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] +Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] -(117) ColumnarToRow [codegen id : 39] -Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +(117) ReusedExchange [Reuses operator id: 58] +Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -(118) SortMergeJoin [codegen id : 40] -Left keys [2]: [ws_order_number#105, ws_item_sk#104] -Right keys [2]: [wr_order_number#118, wr_item_sk#117] -Join type: LeftOuter -Join condition: None +(118) CometSort +Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] + +(119) CometSortMergeJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter -(119) Project [codegen id : 40] -Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#60, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#61] -Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +(120) CometProject +Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] -(120) Union +(121) CometUnion +Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] +Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] -(121) HashAggregate [codegen id : 41] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +(122) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(122) Exchange -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(123) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(123) HashAggregate [codegen id : 42] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +(124) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -Aggregate Attributes: [] -Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(124) HashAggregate [codegen id : 42] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(125) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum#62, sum#121] -Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -(125) Exchange -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(126) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(126) HashAggregate [codegen id : 43] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(127) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] +Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] -Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#20)#66 AS sales_cnt#123, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#124] -(127) Filter [codegen id : 43] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -Condition : isnotnull(sales_cnt#123) +(128) CometFilter +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Condition : isnotnull(sales_cnt#118) + +(129) CometColumnarExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(128) Exchange -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] +(130) CometSort +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] -(129) Sort [codegen id : 44] -Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 +(131) ColumnarToRow [codegen id : 20] +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(130) SortMergeJoin [codegen id : 45] +(132) SortMergeJoin [codegen id : 21] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Right keys [4]: [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Join type: Inner -Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) -(131) Project [codegen id : 45] -Output [10]: [d_year#82 AS prev_year#125, d_year#14 AS year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#123 AS prev_yr_cnt#127, sales_cnt#68 AS curr_yr_cnt#128, (sales_cnt#68 - sales_cnt#123) AS sales_cnt_diff#129, (sales_amt#69 - sales_amt#124) AS sales_amt_diff#130] -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] +(133) Project [codegen id : 21] +Output [10]: [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -(132) TakeOrderedAndProject -Input [10]: [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] -Arguments: 100, [sales_cnt_diff#129 ASC NULLS FIRST, sales_amt_diff#130 ASC NULLS FIRST], [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] +(134) TakeOrderedAndProject +Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] +Arguments: 100, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (136) -+- * ColumnarToRow (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +BroadcastExchange (138) ++- * ColumnarToRow (137) + +- CometFilter (136) + +- CometScan parquet spark_catalog.default.date_dim (135) (unknown) Scan parquet spark_catalog.default.date_dim @@ -744,14 +736,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(134) CometFilter +(136) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(135) ColumnarToRow [codegen id : 1] +(137) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(136) BroadcastExchange +(138) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] @@ -759,33 +751,33 @@ Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 -BroadcastExchange (140) -+- * ColumnarToRow (139) - +- CometFilter (138) - +- CometScan parquet spark_catalog.default.date_dim (137) +Subquery:4 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (142) ++- * ColumnarToRow (141) + +- CometFilter (140) + +- CometScan parquet spark_catalog.default.date_dim (139) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#81, d_year#82] +Output [2]: [d_date_sk#77, d_year#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(138) CometFilter -Input [2]: [d_date_sk#81, d_year#82] -Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) +(140) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(139) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#81, d_year#82] +(141) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#77, d_year#78] -(140) BroadcastExchange -Input [2]: [d_date_sk#81, d_year#82] +(142) BroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 89 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 104 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt index 25dd0f9468..a0e22e72fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt @@ -1,237 +1,185 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt] - WholeStageCodegen (45) + WholeStageCodegen (21) Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter - WholeStageCodegen (22) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (10) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (21) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - WholeStageCodegen (20) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - WholeStageCodegen (19) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - WholeStageCodegen (6) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - InputAdapter - WholeStageCodegen (5) + CometSort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + CometFilter [sales_cnt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometSort [cs_order_number,cs_item_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + RowToColumnar + WholeStageCodegen (3) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [cr_order_number,cr_item_sk] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - WholeStageCodegen (12) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (10) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (9) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] InputAdapter - WholeStageCodegen (11) + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cr_order_number,cr_item_sk] + CometColumnarExchange [cr_order_number,cr_item_sk] #7 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 + RowToColumnar + WholeStageCodegen (6) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [sr_ticket_number,sr_item_sk] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - WholeStageCodegen (18) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - WholeStageCodegen (16) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (15) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter - WholeStageCodegen (17) + ReusedExchange [d_date_sk,d_year] #5 + CometSort [sr_ticket_number,sr_item_sk] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #9 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometSort [ws_order_number,ws_item_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #10 + RowToColumnar + WholeStageCodegen (9) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [wr_order_number,wr_item_sk] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [wr_order_number,wr_item_sk] + CometColumnarExchange [wr_order_number,wr_item_sk] #11 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter - WholeStageCodegen (44) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + WholeStageCodegen (20) + ColumnarToRow InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - WholeStageCodegen (43) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (42) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - WholeStageCodegen (41) - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - InputAdapter - Union - WholeStageCodegen (28) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] - SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - WholeStageCodegen (26) - Sort [cs_order_number,cs_item_sk] - InputAdapter - Exchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (25) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - InputAdapter - WholeStageCodegen (27) + CometSort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + CometFilter [sales_cnt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 + CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometUnion + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometSort [cs_order_number,cs_item_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #15 + RowToColumnar + WholeStageCodegen (13) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [cr_order_number,cr_item_sk] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - WholeStageCodegen (34) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (32) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (31) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter - WholeStageCodegen (33) + ReusedExchange [d_date_sk,d_year] #16 + CometSort [cr_order_number,cr_item_sk] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 + RowToColumnar + WholeStageCodegen (16) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - WholeStageCodegen (40) - Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] - SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter - WholeStageCodegen (38) - Sort [ws_order_number,ws_item_sk] - InputAdapter - Exchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (37) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - InputAdapter - WholeStageCodegen (39) + ReusedExchange [d_date_sk,d_year] #16 + CometSort [sr_ticket_number,sr_item_sk] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometSort [ws_order_number,ws_item_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #18 + RowToColumnar + WholeStageCodegen (19) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometSort [wr_order_number,wr_item_sk] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [wr_order_number,wr_item_sk] + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt index 82c7d9b244..688a02fb0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt @@ -1,102 +1,116 @@ == Physical Plan == -TakeOrderedAndProject (98) -+- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- Union (94) - :- * HashAggregate (83) - : +- Exchange (82) - : +- * HashAggregate (81) - : +- Union (80) - : :- * Project (30) - : : +- * BroadcastHashJoin LeftOuter BuildRight (29) - : : :- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.store (7) - : : +- BroadcastExchange (28) - : : +- * HashAggregate (27) - : : +- Exchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * ColumnarToRow (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.store_returns (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (49) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (48) - : : :- BroadcastExchange (39) - : : : +- * HashAggregate (38) - : : : +- Exchange (37) - : : : +- * HashAggregate (36) - : : : +- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (32) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : : +- ReusedExchange (33) - : : +- * HashAggregate (47) - : : +- Exchange (46) - : : +- * HashAggregate (45) - : : +- * Project (44) - : : +- * BroadcastHashJoin Inner BuildRight (43) - : : :- * ColumnarToRow (41) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (40) - : : +- ReusedExchange (42) - : +- * Project (79) - : +- * BroadcastHashJoin LeftOuter BuildRight (78) - : :- * HashAggregate (64) - : : +- Exchange (63) - : : +- * HashAggregate (62) - : : +- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * ColumnarToRow (52) - : : : : +- CometFilter (51) - : : : : +- CometScan parquet spark_catalog.default.web_sales (50) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.web_page (56) - : +- BroadcastExchange (77) - : +- * HashAggregate (76) - : +- Exchange (75) - : +- * HashAggregate (74) - : +- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * ColumnarToRow (67) - : : : +- CometFilter (66) - : : : +- CometScan parquet spark_catalog.default.web_returns (65) - : : +- ReusedExchange (68) - : +- ReusedExchange (71) - :- * HashAggregate (88) - : +- Exchange (87) - : +- * HashAggregate (86) - : +- * HashAggregate (85) - : +- ReusedExchange (84) - +- * HashAggregate (93) - +- Exchange (92) - +- * HashAggregate (91) - +- * HashAggregate (90) - +- ReusedExchange (89) +* ColumnarToRow (112) ++- CometTakeOrderedAndProject (111) + +- CometHashAggregate (110) + +- CometColumnarExchange (109) + +- CometHashAggregate (108) + +- CometUnion (107) + :- CometHashAggregate (96) + : +- CometColumnarExchange (95) + : +- RowToColumnar (94) + : +- * HashAggregate (93) + : +- Union (92) + : :- * Project (34) + : : +- * BroadcastHashJoin LeftOuter BuildRight (33) + : : :- * ColumnarToRow (17) + : : : +- CometHashAggregate (16) + : : : +- CometColumnarExchange (15) + : : : +- RowToColumnar (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan parquet spark_catalog.default.store (7) + : : +- BroadcastExchange (32) + : : +- * ColumnarToRow (31) + : : +- CometHashAggregate (30) + : : +- CometColumnarExchange (29) + : : +- RowToColumnar (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * ColumnarToRow (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.store_returns (18) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : :- * Project (57) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (56) + : : :- BroadcastExchange (45) + : : : +- * ColumnarToRow (44) + : : : +- CometHashAggregate (43) + : : : +- CometColumnarExchange (42) + : : : +- RowToColumnar (41) + : : : +- * HashAggregate (40) + : : : +- * Project (39) + : : : +- * BroadcastHashJoin Inner BuildRight (38) + : : : :- * ColumnarToRow (36) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (35) + : : : +- ReusedExchange (37) + : : +- * ColumnarToRow (55) + : : +- CometHashAggregate (54) + : : +- CometColumnarExchange (53) + : : +- RowToColumnar (52) + : : +- * HashAggregate (51) + : : +- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * ColumnarToRow (47) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (46) + : : +- ReusedExchange (48) + : +- * Project (91) + : +- * BroadcastHashJoin LeftOuter BuildRight (90) + : :- * ColumnarToRow (74) + : : +- CometHashAggregate (73) + : : +- CometColumnarExchange (72) + : : +- RowToColumnar (71) + : : +- * HashAggregate (70) + : : +- * Project (69) + : : +- * BroadcastHashJoin Inner BuildRight (68) + : : :- * Project (63) + : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : :- * ColumnarToRow (60) + : : : : +- CometFilter (59) + : : : : +- CometScan parquet spark_catalog.default.web_sales (58) + : : : +- ReusedExchange (61) + : : +- BroadcastExchange (67) + : : +- * ColumnarToRow (66) + : : +- CometFilter (65) + : : +- CometScan parquet spark_catalog.default.web_page (64) + : +- BroadcastExchange (89) + : +- * ColumnarToRow (88) + : +- CometHashAggregate (87) + : +- CometColumnarExchange (86) + : +- RowToColumnar (85) + : +- * HashAggregate (84) + : +- * Project (83) + : +- * BroadcastHashJoin Inner BuildRight (82) + : :- * Project (80) + : : +- * BroadcastHashJoin Inner BuildRight (79) + : : :- * ColumnarToRow (77) + : : : +- CometFilter (76) + : : : +- CometScan parquet spark_catalog.default.web_returns (75) + : : +- ReusedExchange (78) + : +- ReusedExchange (81) + :- CometHashAggregate (101) + : +- CometColumnarExchange (100) + : +- CometHashAggregate (99) + : +- CometHashAggregate (98) + : +- ReusedExchange (97) + +- CometHashAggregate (106) + +- CometColumnarExchange (105) + +- CometHashAggregate (104) + +- CometHashAggregate (103) + +- ReusedExchange (102) (unknown) Scan parquet spark_catalog.default.store_sales @@ -114,7 +128,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 3] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 103] +(4) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -162,471 +176,486 @@ Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(Un Aggregate Attributes [2]: [sum#8, sum#9] Results [3]: [s_store_sk#7, sum#10, sum#11] -(14) Exchange +(14) RowToColumnar Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 8] +(15) CometColumnarExchange +Input [3]: [s_store_sk#7, sum#10, sum#11] +Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] + +(17) ColumnarToRow [codegen id : 8] +Input [3]: [s_store_sk#7, sales#12, profit#13] (unknown) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Output [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(sr_returned_date_sk#17 IN dynamicpruning#18)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(17) CometFilter -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) +(19) CometFilter +Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] +Condition : isnotnull(sr_store_sk#14) -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +(20) ColumnarToRow [codegen id : 6] +Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] -(19) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#21] +(21) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#19] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#21] +(22) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_returned_date_sk#17] +Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(21) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] +(23) Project [codegen id : 6] +Output [3]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16] +Input [5]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17, d_date_sk#19] -(22) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#22] +(24) ReusedExchange [Reuses operator id: 10] +Output [1]: [s_store_sk#20] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#22] +(25) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_store_sk#14] +Right keys [1]: [s_store_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] - -(25) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Keys [1]: [s_store_sk#22] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#23, sum#24] -Results [3]: [s_store_sk#22, sum#25, sum#26] - -(26) Exchange -Input [3]: [s_store_sk#22, sum#25, sum#26] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#22, sum#25, sum#26] -Keys [1]: [s_store_sk#22] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] -Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] - -(28) BroadcastExchange -Input [3]: [s_store_sk#22, returns#29, profit_loss#30] +(26) Project [codegen id : 6] +Output [3]: [sr_return_amt#15, sr_net_loss#16, s_store_sk#20] +Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, s_store_sk#20] + +(27) HashAggregate [codegen id : 6] +Input [3]: [sr_return_amt#15, sr_net_loss#16, s_store_sk#20] +Keys [1]: [s_store_sk#20] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#15)), partial_sum(UnscaledValue(sr_net_loss#16))] +Aggregate Attributes [2]: [sum#21, sum#22] +Results [3]: [s_store_sk#20, sum#23, sum#24] + +(28) RowToColumnar +Input [3]: [s_store_sk#20, sum#23, sum#24] + +(29) CometColumnarExchange +Input [3]: [s_store_sk#20, sum#23, sum#24] +Arguments: hashpartitioning(s_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(30) CometHashAggregate +Input [3]: [s_store_sk#20, sum#23, sum#24] +Keys [1]: [s_store_sk#20] +Functions [2]: [sum(UnscaledValue(sr_return_amt#15)), sum(UnscaledValue(sr_net_loss#16))] + +(31) ColumnarToRow [codegen id : 7] +Input [3]: [s_store_sk#20, returns#25, profit_loss#26] + +(32) BroadcastExchange +Input [3]: [s_store_sk#20, returns#25, profit_loss#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 8] +(33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#22] +Right keys [1]: [s_store_sk#20] Join type: LeftOuter Join condition: None -(30) Project [codegen id : 8] -Output [5]: [store channel AS channel#31, s_store_sk#7 AS id#32, sales#14, coalesce(returns#29, 0.00) AS returns#33, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#34] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] +(34) Project [codegen id : 8] +Output [5]: [store channel AS channel#27, s_store_sk#7 AS id#28, sales#12, coalesce(returns#25, 0.00) AS returns#29, (profit#13 - coalesce(profit_loss#26, 0.00)) AS profit#30] +Input [6]: [s_store_sk#7, sales#12, profit#13, s_store_sk#20, returns#25, profit_loss#26] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +Output [4]: [cs_call_center_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#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(36) ColumnarToRow [codegen id : 10] +Input [4]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -(33) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#40] +(37) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#36] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#34] +Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(35) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] - -(36) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum#41, sum#42] -Results [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(37) Exchange -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(38) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] -Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] - -(39) BroadcastExchange -Input [3]: [cs_call_center_sk#35, sales#47, profit#48] +(39) Project [codegen id : 10] +Output [3]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33] +Input [5]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, d_date_sk#36] + +(40) HashAggregate [codegen id : 10] +Input [3]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33] +Keys [1]: [cs_call_center_sk#31] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(UnscaledValue(cs_net_profit#33))] +Aggregate Attributes [2]: [sum#37, sum#38] +Results [3]: [cs_call_center_sk#31, sum#39, sum#40] + +(41) RowToColumnar +Input [3]: [cs_call_center_sk#31, sum#39, sum#40] + +(42) CometColumnarExchange +Input [3]: [cs_call_center_sk#31, sum#39, sum#40] +Arguments: hashpartitioning(cs_call_center_sk#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(43) CometHashAggregate +Input [3]: [cs_call_center_sk#31, sum#39, sum#40] +Keys [1]: [cs_call_center_sk#31] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(UnscaledValue(cs_net_profit#33))] + +(44) ColumnarToRow [codegen id : 11] +Input [3]: [cs_call_center_sk#31, sales#41, profit#42] + +(45) BroadcastExchange +Input [3]: [cs_call_center_sk#31, sales#41, profit#42] Arguments: IdentityBroadcastMode, [plan_id=6] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Output [3]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#46)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(47) ColumnarToRow [codegen id : 13] +Input [3]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -(42) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#53] +(48) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#47] -(43) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#51] -Right keys [1]: [d_date_sk#53] +(49) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cr_returned_date_sk#45] +Right keys [1]: [d_date_sk#47] Join type: Inner Join condition: None -(44) Project [codegen id : 13] -Output [2]: [cr_return_amount#49, cr_net_loss#50] -Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] +(50) Project [codegen id : 13] +Output [2]: [cr_return_amount#43, cr_net_loss#44] +Input [4]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45, d_date_sk#47] -(45) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#49, cr_net_loss#50] +(51) HashAggregate [codegen id : 13] +Input [2]: [cr_return_amount#43, cr_net_loss#44] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum#54, sum#55] -Results [2]: [sum#56, sum#57] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#43)), partial_sum(UnscaledValue(cr_net_loss#44))] +Aggregate Attributes [2]: [sum#48, sum#49] +Results [2]: [sum#50, sum#51] -(46) Exchange -Input [2]: [sum#56, sum#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(52) RowToColumnar +Input [2]: [sum#50, sum#51] -(47) HashAggregate -Input [2]: [sum#56, sum#57] +(53) CometColumnarExchange +Input [2]: [sum#50, sum#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(54) CometHashAggregate +Input [2]: [sum#50, sum#51] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] +Functions [2]: [sum(UnscaledValue(cr_return_amount#43)), sum(UnscaledValue(cr_net_loss#44))] + +(55) ColumnarToRow +Input [2]: [returns#52, profit_loss#53] -(48) BroadcastNestedLoopJoin [codegen id : 14] +(56) BroadcastNestedLoopJoin [codegen id : 14] Join type: Inner Join condition: None -(49) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#62, cs_call_center_sk#35 AS id#63, sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#64] -Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] +(57) Project [codegen id : 14] +Output [5]: [catalog channel AS channel#54, cs_call_center_sk#31 AS id#55, sales#41, returns#52, (profit#42 - profit_loss#53) AS profit#56] +Input [5]: [cs_call_center_sk#31, sales#41, profit#42, returns#52, profit_loss#53] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +Output [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(51) CometFilter -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_web_page_sk#65) +(59) CometFilter +Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] +Condition : isnotnull(ws_web_page_sk#57) -(52) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(60) ColumnarToRow [codegen id : 17] +Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] -(53) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#70] +(61) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#62] -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#68] -Right keys [1]: [d_date_sk#70] +(62) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#60] +Right keys [1]: [d_date_sk#62] Join type: Inner Join condition: None -(55) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] -Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] +(63) Project [codegen id : 17] +Output [3]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59] +Input [5]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60, d_date_sk#62] (unknown) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#71] +Output [1]: [wp_web_page_sk#63] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(57) CometFilter -Input [1]: [wp_web_page_sk#71] -Condition : isnotnull(wp_web_page_sk#71) +(65) CometFilter +Input [1]: [wp_web_page_sk#63] +Condition : isnotnull(wp_web_page_sk#63) -(58) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#71] +(66) ColumnarToRow [codegen id : 16] +Input [1]: [wp_web_page_sk#63] -(59) BroadcastExchange -Input [1]: [wp_web_page_sk#71] +(67) BroadcastExchange +Input [1]: [wp_web_page_sk#63] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#65] -Right keys [1]: [wp_web_page_sk#71] +(68) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_web_page_sk#57] +Right keys [1]: [wp_web_page_sk#63] Join type: Inner Join condition: None -(61) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +(69) Project [codegen id : 17] +Output [3]: [ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] +Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] + +(70) HashAggregate [codegen id : 17] +Input [3]: [ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] +Keys [1]: [wp_web_page_sk#63] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#58)), partial_sum(UnscaledValue(ws_net_profit#59))] +Aggregate Attributes [2]: [sum#64, sum#65] +Results [3]: [wp_web_page_sk#63, sum#66, sum#67] -(62) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum#72, sum#73] -Results [3]: [wp_web_page_sk#71, sum#74, sum#75] +(71) RowToColumnar +Input [3]: [wp_web_page_sk#63, sum#66, sum#67] -(63) Exchange -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(72) CometColumnarExchange +Input [3]: [wp_web_page_sk#63, sum#66, sum#67] +Arguments: hashpartitioning(wp_web_page_sk#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(64) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] -Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] +(73) CometHashAggregate +Input [3]: [wp_web_page_sk#63, sum#66, sum#67] +Keys [1]: [wp_web_page_sk#63] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#58)), sum(UnscaledValue(ws_net_profit#59))] + +(74) ColumnarToRow [codegen id : 22] +Input [3]: [wp_web_page_sk#63, sales#68, profit#69] (unknown) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +Output [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(wr_returned_date_sk#73), dynamicpruningexpression(wr_returned_date_sk#73 IN dynamicpruning#74)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(66) CometFilter -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] -Condition : isnotnull(wr_web_page_sk#80) +(76) CometFilter +Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] +Condition : isnotnull(wr_web_page_sk#70) -(67) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(77) ColumnarToRow [codegen id : 20] +Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] -(68) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#85] +(78) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#75] -(69) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#83] -Right keys [1]: [d_date_sk#85] +(79) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_returned_date_sk#73] +Right keys [1]: [d_date_sk#75] Join type: Inner Join condition: None -(70) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] -Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] +(80) Project [codegen id : 20] +Output [3]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72] +Input [5]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73, d_date_sk#75] -(71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#86] +(81) ReusedExchange [Reuses operator id: 67] +Output [1]: [wp_web_page_sk#76] -(72) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#80] -Right keys [1]: [wp_web_page_sk#86] +(82) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_web_page_sk#70] +Right keys [1]: [wp_web_page_sk#76] Join type: Inner Join condition: None -(73) Project [codegen id : 20] -Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] - -(74) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum#87, sum#88] -Results [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(75) Exchange -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(76) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] -Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] - -(77) BroadcastExchange -Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] +(83) Project [codegen id : 20] +Output [3]: [wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] +Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] + +(84) HashAggregate [codegen id : 20] +Input [3]: [wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] +Keys [1]: [wp_web_page_sk#76] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#71)), partial_sum(UnscaledValue(wr_net_loss#72))] +Aggregate Attributes [2]: [sum#77, sum#78] +Results [3]: [wp_web_page_sk#76, sum#79, sum#80] + +(85) RowToColumnar +Input [3]: [wp_web_page_sk#76, sum#79, sum#80] + +(86) CometColumnarExchange +Input [3]: [wp_web_page_sk#76, sum#79, sum#80] +Arguments: hashpartitioning(wp_web_page_sk#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(87) CometHashAggregate +Input [3]: [wp_web_page_sk#76, sum#79, sum#80] +Keys [1]: [wp_web_page_sk#76] +Functions [2]: [sum(UnscaledValue(wr_return_amt#71)), sum(UnscaledValue(wr_net_loss#72))] + +(88) ColumnarToRow [codegen id : 21] +Input [3]: [wp_web_page_sk#76, returns#81, profit_loss#82] + +(89) BroadcastExchange +Input [3]: [wp_web_page_sk#76, returns#81, profit_loss#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(78) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#71] -Right keys [1]: [wp_web_page_sk#86] +(90) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [wp_web_page_sk#63] +Right keys [1]: [wp_web_page_sk#76] Join type: LeftOuter Join condition: None -(79) Project [codegen id : 22] -Output [5]: [web channel AS channel#95, wp_web_page_sk#71 AS id#96, sales#78, coalesce(returns#93, 0.00) AS returns#97, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#98] -Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] - -(80) Union - -(81) HashAggregate [codegen id : 23] -Input [5]: [channel#31, id#32, sales#14, returns#33, profit#34] -Keys [2]: [channel#31, id#32] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#33), partial_sum(profit#34)] -Aggregate Attributes [6]: [sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(82) Exchange -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(83) HashAggregate [codegen id : 24] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [5]: [channel#31, id#32, cast(sum(sales#14)#111 as decimal(37,2)) AS sales#114, cast(sum(returns#33)#112 as decimal(37,2)) AS returns#115, cast(sum(profit#34)#113 as decimal(38,2)) AS profit#116] - -(84) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(85) HashAggregate [codegen id : 48] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [4]: [channel#31, sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] - -(86) HashAggregate [codegen id : 48] -Input [4]: [channel#31, sales#117, returns#118, profit#119] -Keys [1]: [channel#31] -Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] -Aggregate Attributes [6]: [sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Results [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(87) Exchange -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(88) HashAggregate [codegen id : 49] -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Keys [1]: [channel#31] -Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] -Aggregate Attributes [3]: [sum(sales#117)#132, sum(returns#118)#133, sum(profit#119)#134] -Results [5]: [channel#31, null AS id#135, sum(sales#117)#132 AS sales#136, sum(returns#118)#133 AS returns#137, sum(profit#119)#134 AS profit#138] - -(89) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(90) HashAggregate [codegen id : 73] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [3]: [sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] - -(91) HashAggregate [codegen id : 73] -Input [3]: [sales#117, returns#118, profit#119] +(91) Project [codegen id : 22] +Output [5]: [web channel AS channel#83, wp_web_page_sk#63 AS id#84, sales#68, coalesce(returns#81, 0.00) AS returns#85, (profit#69 - coalesce(profit_loss#82, 0.00)) AS profit#86] +Input [6]: [wp_web_page_sk#63, sales#68, profit#69, wp_web_page_sk#76, returns#81, profit_loss#82] + +(92) Union + +(93) HashAggregate [codegen id : 23] +Input [5]: [channel#27, id#28, sales#12, returns#29, profit#30] +Keys [2]: [channel#27, id#28] +Functions [3]: [partial_sum(sales#12), partial_sum(returns#29), partial_sum(profit#30)] +Aggregate Attributes [6]: [sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92] +Results [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] + +(94) RowToColumnar +Input [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] + +(95) CometColumnarExchange +Input [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Arguments: hashpartitioning(channel#27, id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(96) CometHashAggregate +Input [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Keys [2]: [channel#27, id#28] +Functions [3]: [sum(sales#12), sum(returns#29), sum(profit#30)] + +(97) ReusedExchange [Reuses operator id: 95] +Output [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] + +(98) CometHashAggregate +Input [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Keys [2]: [channel#27, id#28] +Functions [3]: [sum(sales#12), sum(returns#29), sum(profit#30)] + +(99) CometHashAggregate +Input [4]: [channel#27, sales#99, returns#100, profit#101] +Keys [1]: [channel#27] +Functions [3]: [partial_sum(sales#99), partial_sum(returns#100), partial_sum(profit#101)] + +(100) CometColumnarExchange +Input [7]: [channel#27, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] +Arguments: hashpartitioning(channel#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(101) CometHashAggregate +Input [7]: [channel#27, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] +Keys [1]: [channel#27] +Functions [3]: [sum(sales#99), sum(returns#100), sum(profit#101)] + +(102) ReusedExchange [Reuses operator id: 95] +Output [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] + +(103) CometHashAggregate +Input [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Keys [2]: [channel#27, id#28] +Functions [3]: [sum(sales#12), sum(returns#29), sum(profit#30)] + +(104) CometHashAggregate +Input [3]: [sales#99, returns#100, profit#101] Keys: [] -Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] -Aggregate Attributes [6]: [sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -Results [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] +Functions [3]: [partial_sum(sales#99), partial_sum(returns#100), partial_sum(profit#101)] -(92) Exchange -Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] +(105) CometColumnarExchange +Input [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(93) HashAggregate [codegen id : 74] -Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] +(106) CometHashAggregate +Input [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] Keys: [] -Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] -Aggregate Attributes [3]: [sum(sales#117)#151, sum(returns#118)#152, sum(profit#119)#153] -Results [5]: [null AS channel#154, null AS id#155, sum(sales#117)#151 AS sales#156, sum(returns#118)#152 AS returns#157, sum(profit#119)#153 AS profit#158] +Functions [3]: [sum(sales#99), sum(returns#100), sum(profit#101)] -(94) Union +(107) CometUnion +Child 0 Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] +Child 1 Input [5]: [channel#27, id#117, sales#118, returns#119, profit#120] +Child 2 Input [5]: [channel#121, id#122, sales#123, returns#124, profit#125] -(95) HashAggregate [codegen id : 75] -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(108) CometHashAggregate +Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] +Keys [5]: [channel#27, id#28, sales#114, returns#115, profit#116] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -(96) Exchange -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: hashpartitioning(channel#31, id#32, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(109) CometColumnarExchange +Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] +Arguments: hashpartitioning(channel#27, id#28, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(97) HashAggregate [codegen id : 76] -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(110) CometHashAggregate +Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] +Keys [5]: [channel#27, id#28, sales#114, returns#115, profit#116] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -(98) TakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: 100, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#114, returns#115, profit#116] +(111) CometTakeOrderedAndProject +Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,id#28 ASC NULLS FIRST], output=[channel#27,id#28,sales#114,returns#115,profit#116]), 100, [channel#27 ASC NULLS FIRST, id#28 ASC NULLS FIRST], [channel#27, id#28, sales#114, returns#115, profit#116] + +(112) ColumnarToRow [codegen id : 70] +Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (103) -+- * ColumnarToRow (102) - +- CometProject (101) - +- CometFilter (100) - +- CometScan parquet spark_catalog.default.date_dim (99) +BroadcastExchange (117) ++- * ColumnarToRow (116) + +- CometProject (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#159] +Output [2]: [d_date_sk#6, d_date#126] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(100) CometFilter -Input [2]: [d_date_sk#6, d_date#159] -Condition : (((isnotnull(d_date#159) AND (d_date#159 >= 1998-08-04)) AND (d_date#159 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +(114) CometFilter +Input [2]: [d_date_sk#6, d_date#126] +Condition : (((isnotnull(d_date#126) AND (d_date#126 >= 1998-08-04)) AND (d_date#126 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(101) CometProject -Input [2]: [d_date_sk#6, d_date#159] +(115) CometProject +Input [2]: [d_date_sk#6, d_date#126] Arguments: [d_date_sk#6], [d_date_sk#6] -(102) ColumnarToRow [codegen id : 1] +(116) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(103) BroadcastExchange +(117) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 40 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 46 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 65 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 75 Hosting Expression = wr_returned_date_sk#73 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index 670a7e6c3e..1fc58ce481 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -1,26 +1,26 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (76) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (24) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [channel,id] #2 - WholeStageCodegen (23) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - InputAdapter - Exchange [s_store_sk] #3 +WholeStageCodegen (70) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (23) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (8) + Project [s_store_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [s_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_sk,sum,sum] + CometColumnarExchange [s_store_sk] #3 + RowToColumnar WholeStageCodegen (3) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] Project [ss_ext_sales_price,ss_net_profit,s_store_sk] @@ -48,12 +48,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - InputAdapter - Exchange [s_store_sk] #7 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_sk,sum,sum] + CometColumnarExchange [s_store_sk] #7 + RowToColumnar WholeStageCodegen (6) HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] Project [sr_return_amt,sr_net_loss,s_store_sk] @@ -69,15 +71,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #4 InputAdapter ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - InputAdapter - Exchange [cs_call_center_sk] #9 + WholeStageCodegen (14) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometHashAggregate [cs_call_center_sk,sum,sum] + CometColumnarExchange [cs_call_center_sk] #9 + RowToColumnar WholeStageCodegen (10) HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] @@ -88,9 +92,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - InputAdapter - Exchange #10 + ColumnarToRow + InputAdapter + CometHashAggregate [sum,sum] + CometColumnarExchange #10 + RowToColumnar WholeStageCodegen (13) HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] Project [cr_return_amount,cr_net_loss] @@ -101,12 +107,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - InputAdapter - Exchange [wp_web_page_sk] #11 + WholeStageCodegen (22) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + ColumnarToRow + InputAdapter + CometHashAggregate [wp_web_page_sk,sum,sum] + CometColumnarExchange [wp_web_page_sk] #11 + RowToColumnar WholeStageCodegen (17) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] @@ -127,12 +135,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - InputAdapter - Exchange [wp_web_page_sk] #14 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (21) + ColumnarToRow + InputAdapter + CometHashAggregate [wp_web_page_sk,sum,sum] + CometColumnarExchange [wp_web_page_sk] #14 + RowToColumnar WholeStageCodegen (20) HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] Project [wr_return_amt,wr_net_loss,wp_web_page_sk] @@ -148,21 +158,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #4 InputAdapter ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [channel] #15 - WholeStageCodegen (48) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange #16 - WholeStageCodegen (73) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [channel] #15 + CometHashAggregate [channel,sales,returns,profit] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange #16 + CometHashAggregate [sales,returns,profit] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index cfb9ed7d0e..f411b80b79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -1,71 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * SortMergeJoin Inner (65) - :- * Project (43) - : +- * SortMergeJoin Inner (42) - : :- * Sort (20) - : : +- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (14) - : +- * Sort (41) - : +- * Filter (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometSortMergeJoin (30) - : : :- CometSort (24) - : : : +- CometExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan parquet spark_catalog.default.web_sales (21) - : : +- CometSort (29) - : : +- CometExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan parquet spark_catalog.default.web_returns (25) - : +- ReusedExchange (34) - +- * Sort (64) - +- * Filter (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * ColumnarToRow (56) - : +- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (57) +TakeOrderedAndProject (71) ++- * Project (70) + +- * ColumnarToRow (69) + +- CometSortMergeJoin (68) + :- CometProject (45) + : +- CometSortMergeJoin (44) + : :- CometSort (21) + : : +- CometHashAggregate (20) + : : +- CometColumnarExchange (19) + : : +- RowToColumnar (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * ColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : +- ReusedExchange (14) + : +- CometSort (43) + : +- CometFilter (42) + : +- CometHashAggregate (41) + : +- CometColumnarExchange (40) + : +- RowToColumnar (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * ColumnarToRow (34) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometSortMergeJoin (31) + : : :- CometSort (25) + : : : +- CometColumnarExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.web_returns (26) + : +- ReusedExchange (35) + +- CometSort (67) + +- CometFilter (66) + +- CometHashAggregate (65) + +- CometColumnarExchange (64) + +- RowToColumnar (63) + +- * HashAggregate (62) + +- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * ColumnarToRow (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometColumnarExchange (48) + : : +- CometFilter (47) + : : +- CometScan parquet spark_catalog.default.catalog_sales (46) + : +- CometSort (54) + : +- CometColumnarExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.catalog_returns (50) + +- ReusedExchange (59) (unknown) Scan parquet spark_catalog.default.store_sales @@ -80,9 +84,9 @@ ReadSchema: struct -(22) CometFilter -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) +(23) CometFilter +Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +Condition : (isnotnull(ws_item_sk#24) AND isnotnull(ws_bill_customer_sk#25)) -(23) CometExchange -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(24) CometColumnarExchange +Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +Arguments: hashpartitioning(ws_order_number#26, ws_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometSort -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] +(25) CometSort +Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_order_number#26 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Output [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(26) CometFilter -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) +(27) CometFilter +Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] +Condition : (isnotnull(wr_order_number#33) AND isnotnull(wr_item_sk#32)) -(27) CometProject -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] +(28) CometProject +Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] +Arguments: [wr_item_sk#32, wr_order_number#33], [wr_item_sk#32, wr_order_number#33] -(28) CometExchange -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(29) CometColumnarExchange +Input [2]: [wr_item_sk#32, wr_order_number#33] +Arguments: hashpartitioning(wr_order_number#33, wr_item_sk#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(29) CometSort -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] +(30) CometSort +Input [2]: [wr_item_sk#32, wr_order_number#33] +Arguments: [wr_item_sk#32, wr_order_number#33], [wr_order_number#33 ASC NULLS FIRST, wr_item_sk#32 ASC NULLS FIRST] -(30) CometSortMergeJoin -Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Right output [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter +(31) CometSortMergeJoin +Left output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +Right output [2]: [wr_item_sk#32, wr_order_number#33] +Arguments: [ws_order_number#26, ws_item_sk#24], [wr_order_number#33, wr_item_sk#32], LeftOuter -(31) CometFilter -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Condition : isnull(wr_order_number#36) +(32) CometFilter +Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] +Condition : isnull(wr_order_number#33) -(32) CometProject -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +(33) CometProject +Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] +Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -(33) ColumnarToRow [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +(34) ColumnarToRow [codegen id : 4] +Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -(34) ReusedExchange [Reuses operator id: 71] -Output [2]: [d_date_sk#38, d_year#39] +(35) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#35, d_year#36] -(35) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] +(36) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#35] Join type: Inner Join condition: None -(36) Project [codegen id : 5] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] - -(37) HashAggregate [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum#40, sum#41, sum#42] -Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(38) Exchange -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(39) HashAggregate [codegen id : 6] -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] -Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] - -(40) Filter [codegen id : 6] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Condition : (coalesce(ws_qty#51, 0) > 0) - -(41) Sort [codegen id : 6] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 - -(42) SortMergeJoin [codegen id : 7] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] -Join type: Inner -Join condition: None +(37) Project [codegen id : 4] +Output [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] +Input [8]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, d_date_sk#35, d_year#36] + +(38) HashAggregate [codegen id : 4] +Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] +Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] +Functions [3]: [partial_sum(ws_quantity#27), partial_sum(UnscaledValue(ws_wholesale_cost#28)), partial_sum(UnscaledValue(ws_sales_price#29))] +Aggregate Attributes [3]: [sum#37, sum#38, sum#39] +Results [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] + +(39) RowToColumnar +Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] + +(40) CometColumnarExchange +Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] +Arguments: hashpartitioning(d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(41) CometHashAggregate +Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] +Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] +Functions [3]: [sum(ws_quantity#27), sum(UnscaledValue(ws_wholesale_cost#28)), sum(UnscaledValue(ws_sales_price#29))] + +(42) CometFilter +Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] +Condition : (coalesce(ws_qty#45, 0) > 0) + +(43) CometSort +Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] +Arguments: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47], [ws_sold_year#43 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST, ws_customer_sk#44 ASC NULLS FIRST] + +(44) CometSortMergeJoin +Left output [6]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] +Right output [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] +Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44], Inner -(43) Project [codegen id : 7] -Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] -Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +(45) CometProject +Input [12]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] +Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47], [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#54), dynamicpruningexpression(cs_sold_date_sk#54 IN dynamicpruning#55)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(45) CometFilter -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) +(47) CometFilter +Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +Condition : (isnotnull(cs_item_sk#49) AND isnotnull(cs_bill_customer_sk#48)) -(46) CometExchange -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(48) CometColumnarExchange +Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +Arguments: hashpartitioning(cs_order_number#50, cs_item_sk#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(47) CometSort -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] +(49) CometSort +Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_order_number#50 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Output [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(49) CometFilter -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) +(51) CometFilter +Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] +Condition : (isnotnull(cr_order_number#57) AND isnotnull(cr_item_sk#56)) -(50) CometProject -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] +(52) CometProject +Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] +Arguments: [cr_item_sk#56, cr_order_number#57], [cr_item_sk#56, cr_order_number#57] -(51) CometExchange -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(53) CometColumnarExchange +Input [2]: [cr_item_sk#56, cr_order_number#57] +Arguments: hashpartitioning(cr_order_number#57, cr_item_sk#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(52) CometSort -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] +(54) CometSort +Input [2]: [cr_item_sk#56, cr_order_number#57] +Arguments: [cr_item_sk#56, cr_order_number#57], [cr_order_number#57 ASC NULLS FIRST, cr_item_sk#56 ASC NULLS FIRST] -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Right output [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter +(55) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +Right output [2]: [cr_item_sk#56, cr_order_number#57] +Arguments: [cs_order_number#50, cs_item_sk#49], [cr_order_number#57, cr_item_sk#56], LeftOuter -(54) CometFilter -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Condition : isnull(cr_order_number#63) +(56) CometFilter +Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] +Condition : isnull(cr_order_number#57) -(55) CometProject -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(57) CometProject +Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] +Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -(56) ColumnarToRow [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(58) ColumnarToRow [codegen id : 6] +Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -(57) ReusedExchange [Reuses operator id: 71] -Output [2]: [d_date_sk#65, d_year#66] +(59) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#59, d_year#60] -(58) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#65] +(60) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#54] +Right keys [1]: [d_date_sk#59] Join type: Inner Join condition: None -(59) Project [codegen id : 9] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] - -(60) HashAggregate [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum#67, sum#68, sum#69] -Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] - -(61) Exchange -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(62) HashAggregate [codegen id : 10] -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] -Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] - -(63) Filter [codegen id : 10] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Condition : (coalesce(cs_qty#78, 0) > 0) - -(64) Sort [codegen id : 10] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 - -(65) SortMergeJoin [codegen id : 11] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] -Join type: Inner -Join condition: None +(61) Project [codegen id : 6] +Output [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] +Input [8]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, d_date_sk#59, d_year#60] + +(62) HashAggregate [codegen id : 6] +Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] +Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] +Functions [3]: [partial_sum(cs_quantity#51), partial_sum(UnscaledValue(cs_wholesale_cost#52)), partial_sum(UnscaledValue(cs_sales_price#53))] +Aggregate Attributes [3]: [sum#61, sum#62, sum#63] +Results [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] + +(63) RowToColumnar +Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] + +(64) CometColumnarExchange +Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] +Arguments: hashpartitioning(d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(65) CometHashAggregate +Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] +Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] +Functions [3]: [sum(cs_quantity#51), sum(UnscaledValue(cs_wholesale_cost#52)), sum(UnscaledValue(cs_sales_price#53))] + +(66) CometFilter +Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] +Condition : (coalesce(cs_qty#69, 0) > 0) + +(67) CometSort +Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] +Arguments: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71], [cs_sold_year#67 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST, cs_customer_sk#68 ASC NULLS FIRST] + +(68) CometSortMergeJoin +Left output [9]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] +Right output [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] +Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68], Inner + +(69) ColumnarToRow [codegen id : 7] +Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -(66) Project [codegen id : 11] -Output [13]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +(70) Project [codegen id : 7] +Output [13]: [round((cast(ss_qty#21 as double) / cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)), 2) AS ratio#72, ss_qty#21 AS store_qty#73, ss_wc#22 AS store_wholesale_cost#74, ss_sp#23 AS store_sales_price#75, (coalesce(ws_qty#45, 0) + coalesce(cs_qty#69, 0)) AS other_chan_qty#76, (coalesce(ws_wc#46, 0.00) + coalesce(cs_wc#70, 0.00)) AS other_chan_wholesale_cost#77, (coalesce(ws_sp#47, 0.00) + coalesce(cs_sp#71, 0.00)) AS other_chan_sales_price#78, ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] +Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -(67) TakeOrderedAndProject -Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Arguments: 100, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] +(71) TakeOrderedAndProject +Input [13]: [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78, ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] +Arguments: 100, [ss_sold_year#20 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#21 DESC NULLS LAST, ss_wc#22 DESC NULLS LAST, ss_sp#23 DESC NULLS LAST, other_chan_qty#76 ASC NULLS FIRST, other_chan_wholesale_cost#77 ASC NULLS FIRST, other_chan_sales_price#78 ASC NULLS FIRST, ratio#72 ASC NULLS FIRST], [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * ColumnarToRow (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim @@ -398,19 +406,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter +(73) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(71) BroadcastExchange +(75) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 21 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#54 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index ea1de2fd73..1b83b3157b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -1,80 +1,74 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (11) + WholeStageCodegen (7) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (7) - Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_sold_year,ss_item_sk,ss_customer_sk] - HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (2) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [sr_ticket_number] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_ticket_number,sr_item_sk] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] + CometSort [ss_sold_year,ss_item_sk,ss_customer_sk] + CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [sr_ticket_number] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_ticket_number,sr_item_sk] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + CometSort [ws_sold_year,ws_item_sk,ws_customer_sk] + CometFilter [ws_qty] + CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (6) - Sort [ws_sold_year,ws_item_sk,ws_customer_sk] - Filter [ws_qty] - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (5) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [wr_order_number] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometExchange [ws_order_number,ws_item_sk] #6 - CometFilter [ws_item_sk,ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_order_number,wr_item_sk] - CometExchange [wr_order_number,wr_item_sk] #7 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (10) - Sort [cs_sold_year,cs_item_sk,cs_customer_sk] - Filter [cs_qty] - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - InputAdapter - Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (9) + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [wr_order_number] + CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometSort [ws_order_number,ws_item_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #6 + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_order_number,wr_item_sk] + CometColumnarExchange [wr_order_number,wr_item_sk] #7 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + CometSort [cs_sold_year,cs_item_sk,cs_customer_sk] + CometFilter [cs_qty] + CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 + RowToColumnar + WholeStageCodegen (6) HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -84,12 +78,12 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp CometFilter [cr_order_number] CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] CometSort [cs_order_number,cs_item_sk] - CometExchange [cs_order_number,cs_item_sk] #9 + CometColumnarExchange [cs_order_number,cs_item_sk] #9 CometFilter [cs_item_sk,cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometSort [cr_order_number,cr_item_sk] - CometExchange [cr_order_number,cr_item_sk] #10 + CometColumnarExchange [cr_order_number,cr_item_sk] #10 CometProject [cr_item_sk,cr_order_number] CometFilter [cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] 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 dfd081b442..71122484d8 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,121 +1,129 @@ == Physical Plan == -TakeOrderedAndProject (117) -+- * HashAggregate (116) - +- Exchange (115) - +- * HashAggregate (114) - +- Union (113) - :- * HashAggregate (102) - : +- Exchange (101) - : +- * HashAggregate (100) - : +- Union (99) - : :- * HashAggregate (38) - : : +- Exchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * Project (21) - : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : : : : +- ReusedExchange (13) - : : : : +- BroadcastExchange (19) - : : : : +- * ColumnarToRow (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.store (16) - : : : +- BroadcastExchange (26) - : : : +- * ColumnarToRow (25) - : : : +- CometProject (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.item (22) - : : +- BroadcastExchange (33) - : : +- * ColumnarToRow (32) - : : +- CometProject (31) - : : +- CometFilter (30) - : : +- CometScan parquet spark_catalog.default.promotion (29) - : :- * HashAggregate (68) - : : +- Exchange (67) - : : +- * HashAggregate (66) - : : +- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Project (62) - : : : +- * BroadcastHashJoin Inner BuildRight (61) - : : : :- * Project (59) - : : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : : :- * Project (53) - : : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : : :- * ColumnarToRow (50) - : : : : : : +- CometProject (49) - : : : : : : +- CometSortMergeJoin (48) - : : : : : : :- CometSort (42) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (46) - : : : : : : +- CometProject (45) - : : : : : : +- CometFilter (44) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (43) - : : : : : +- ReusedExchange (51) - : : : : +- BroadcastExchange (57) - : : : : +- * ColumnarToRow (56) - : : : : +- CometFilter (55) - : : : : +- CometScan parquet spark_catalog.default.catalog_page (54) - : : : +- ReusedExchange (60) - : : +- ReusedExchange (63) - : +- * HashAggregate (98) - : +- Exchange (97) - : +- * HashAggregate (96) - : +- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (89) - : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : :- * Project (83) - : : : : +- * BroadcastHashJoin Inner BuildRight (82) - : : : : :- * ColumnarToRow (80) - : : : : : +- CometProject (79) - : : : : : +- CometSortMergeJoin (78) - : : : : : :- CometSort (72) - : : : : : : +- CometExchange (71) - : : : : : : +- CometFilter (70) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (69) - : : : : : +- CometSort (77) - : : : : : +- CometExchange (76) - : : : : : +- CometProject (75) - : : : : : +- CometFilter (74) - : : : : : +- CometScan parquet spark_catalog.default.web_returns (73) - : : : : +- ReusedExchange (81) - : : : +- BroadcastExchange (87) - : : : +- * ColumnarToRow (86) - : : : +- CometFilter (85) - : : : +- CometScan parquet spark_catalog.default.web_site (84) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (107) - : +- Exchange (106) - : +- * HashAggregate (105) - : +- * HashAggregate (104) - : +- ReusedExchange (103) - +- * HashAggregate (112) - +- Exchange (111) - +- * HashAggregate (110) - +- * HashAggregate (109) - +- ReusedExchange (108) +* ColumnarToRow (125) ++- CometTakeOrderedAndProject (124) + +- CometHashAggregate (123) + +- CometColumnarExchange (122) + +- CometHashAggregate (121) + +- CometUnion (120) + :- CometHashAggregate (109) + : +- CometColumnarExchange (108) + : +- RowToColumnar (107) + : +- * HashAggregate (106) + : +- Union (105) + : :- * HashAggregate (40) + : : +- * ColumnarToRow (39) + : : +- CometColumnarExchange (38) + : : +- RowToColumnar (37) + : : +- * HashAggregate (36) + : : +- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Project (28) + : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : :- * Project (21) + : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometColumnarExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometColumnarExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : : : : +- ReusedExchange (13) + : : : : +- BroadcastExchange (19) + : : : : +- * ColumnarToRow (18) + : : : : +- CometFilter (17) + : : : : +- CometScan parquet spark_catalog.default.store (16) + : : : +- BroadcastExchange (26) + : : : +- * ColumnarToRow (25) + : : : +- CometProject (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.item (22) + : : +- BroadcastExchange (33) + : : +- * ColumnarToRow (32) + : : +- CometProject (31) + : : +- CometFilter (30) + : : +- CometScan parquet spark_catalog.default.promotion (29) + : :- * HashAggregate (72) + : : +- * ColumnarToRow (71) + : : +- CometColumnarExchange (70) + : : +- RowToColumnar (69) + : : +- * HashAggregate (68) + : : +- * Project (67) + : : +- * BroadcastHashJoin Inner BuildRight (66) + : : :- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : :- * Project (61) + : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : :- * Project (55) + : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : :- * ColumnarToRow (52) + : : : : : : +- CometProject (51) + : : : : : : +- CometSortMergeJoin (50) + : : : : : : :- CometSort (44) + : : : : : : : +- CometColumnarExchange (43) + : : : : : : : +- CometFilter (42) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) + : : : : : : +- CometSort (49) + : : : : : : +- CometColumnarExchange (48) + : : : : : : +- CometProject (47) + : : : : : : +- CometFilter (46) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) + : : : : : +- ReusedExchange (53) + : : : : +- BroadcastExchange (59) + : : : : +- * ColumnarToRow (58) + : : : : +- CometFilter (57) + : : : : +- CometScan parquet spark_catalog.default.catalog_page (56) + : : : +- ReusedExchange (62) + : : +- ReusedExchange (65) + : +- * HashAggregate (104) + : +- * ColumnarToRow (103) + : +- CometColumnarExchange (102) + : +- RowToColumnar (101) + : +- * HashAggregate (100) + : +- * Project (99) + : +- * BroadcastHashJoin Inner BuildRight (98) + : :- * Project (96) + : : +- * BroadcastHashJoin Inner BuildRight (95) + : : :- * Project (93) + : : : +- * BroadcastHashJoin Inner BuildRight (92) + : : : :- * Project (87) + : : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : : :- * ColumnarToRow (84) + : : : : : +- CometProject (83) + : : : : : +- CometSortMergeJoin (82) + : : : : : :- CometSort (76) + : : : : : : +- CometColumnarExchange (75) + : : : : : : +- CometFilter (74) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (73) + : : : : : +- CometSort (81) + : : : : : +- CometColumnarExchange (80) + : : : : : +- CometProject (79) + : : : : : +- CometFilter (78) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (77) + : : : : +- ReusedExchange (85) + : : : +- BroadcastExchange (91) + : : : +- * ColumnarToRow (90) + : : : +- CometFilter (89) + : : : +- CometScan parquet spark_catalog.default.web_site (88) + : : +- ReusedExchange (94) + : +- ReusedExchange (97) + :- CometHashAggregate (114) + : +- CometColumnarExchange (113) + : +- CometHashAggregate (112) + : +- CometHashAggregate (111) + : +- ReusedExchange (110) + +- CometHashAggregate (119) + +- CometColumnarExchange (118) + +- CometHashAggregate (117) + +- CometHashAggregate (116) + +- ReusedExchange (115) (unknown) Scan parquet spark_catalog.default.store_sales @@ -130,9 +138,9 @@ ReadSchema: struct -(40) CometFilter +(42) CometFilter Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) -(41) CometExchange +(43) CometColumnarExchange Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(42) CometSort +(44) CometSort Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45], [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST] @@ -323,44 +337,44 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(44) CometFilter +(46) CometFilter Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) -(45) CometProject +(47) CometProject Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -(46) CometExchange +(48) CometColumnarExchange Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(47) CometSort +(49) CometSort Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST] -(48) CometSortMergeJoin +(50) CometSortMergeJoin Left output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Right output [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] Arguments: [cs_item_sk#40, cs_order_number#42], [cr_item_sk#47, cr_order_number#48], LeftOuter -(49) CometProject +(51) CometProject Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50], [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -(50) ColumnarToRow [codegen id : 11] +(52) ColumnarToRow [codegen id : 11] Input [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -(51) ReusedExchange [Reuses operator id: 122] +(53) ReusedExchange [Reuses operator id: 130] Output [1]: [d_date_sk#52] -(52) BroadcastHashJoin [codegen id : 11] +(54) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#45] Right keys [1]: [d_date_sk#52] Join type: Inner Join condition: None -(53) Project [codegen id : 11] +(55) Project [codegen id : 11] Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] @@ -371,65 +385,71 @@ Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter +(57) CometFilter Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Condition : isnotnull(cp_catalog_page_sk#53) -(56) ColumnarToRow [codegen id : 8] +(58) ColumnarToRow [codegen id : 8] Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -(57) BroadcastExchange +(59) BroadcastExchange Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(58) BroadcastHashJoin [codegen id : 11] +(60) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_catalog_page_sk#39] Right keys [1]: [cp_catalog_page_sk#53] Join type: Inner Join condition: None -(59) Project [codegen id : 11] +(61) Project [codegen id : 11] Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] -(60) ReusedExchange [Reuses operator id: 26] +(62) ReusedExchange [Reuses operator id: 26] Output [1]: [i_item_sk#55] -(61) BroadcastHashJoin [codegen id : 11] +(63) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#40] Right keys [1]: [i_item_sk#55] Join type: Inner Join condition: None -(62) Project [codegen id : 11] +(64) Project [codegen id : 11] Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] -(63) ReusedExchange [Reuses operator id: 33] +(65) ReusedExchange [Reuses operator id: 33] Output [1]: [p_promo_sk#56] -(64) BroadcastHashJoin [codegen id : 11] +(66) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_promo_sk#41] Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(65) Project [codegen id : 11] +(67) Project [codegen id : 11] Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] -(66) HashAggregate [codegen id : 11] +(68) HashAggregate [codegen id : 11] Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -(67) Exchange +(69) RowToColumnar +Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] + +(70) CometColumnarExchange +Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(71) ColumnarToRow [codegen id : 12] Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(68) HashAggregate [codegen id : 12] +(72) HashAggregate [codegen id : 12] Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] @@ -444,15 +464,15 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(70) CometFilter +(74) CometFilter Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) -(71) CometExchange +(75) CometColumnarExchange Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(72) CometSort +(76) CometSort Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81], [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST] @@ -463,44 +483,44 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(74) CometFilter +(78) CometFilter Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) -(75) CometProject +(79) CometProject Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -(76) CometExchange +(80) CometColumnarExchange Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(77) CometSort +(81) CometSort Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST] -(78) CometSortMergeJoin +(82) CometSortMergeJoin Left output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] Right output [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] Arguments: [ws_item_sk#75, ws_order_number#78], [wr_item_sk#83, wr_order_number#84], LeftOuter -(79) CometProject +(83) CometProject Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86], [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -(80) ColumnarToRow [codegen id : 17] +(84) ColumnarToRow [codegen id : 17] Input [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -(81) ReusedExchange [Reuses operator id: 122] +(85) ReusedExchange [Reuses operator id: 130] Output [1]: [d_date_sk#88] -(82) BroadcastHashJoin [codegen id : 17] +(86) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_sold_date_sk#81] Right keys [1]: [d_date_sk#88] Join type: Inner Join condition: None -(83) Project [codegen id : 17] +(87) Project [codegen id : 17] Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] @@ -511,205 +531,202 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter +(89) CometFilter Input [2]: [web_site_sk#89, web_site_id#90] Condition : isnotnull(web_site_sk#89) -(86) ColumnarToRow [codegen id : 14] +(90) ColumnarToRow [codegen id : 14] Input [2]: [web_site_sk#89, web_site_id#90] -(87) BroadcastExchange +(91) BroadcastExchange Input [2]: [web_site_sk#89, web_site_id#90] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(88) BroadcastHashJoin [codegen id : 17] +(92) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_web_site_sk#76] Right keys [1]: [web_site_sk#89] Join type: Inner Join condition: None -(89) Project [codegen id : 17] +(93) Project [codegen id : 17] Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] -(90) ReusedExchange [Reuses operator id: 26] +(94) ReusedExchange [Reuses operator id: 26] Output [1]: [i_item_sk#91] -(91) BroadcastHashJoin [codegen id : 17] +(95) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_item_sk#75] Right keys [1]: [i_item_sk#91] Join type: Inner Join condition: None -(92) Project [codegen id : 17] +(96) Project [codegen id : 17] Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] -(93) ReusedExchange [Reuses operator id: 33] +(97) ReusedExchange [Reuses operator id: 33] Output [1]: [p_promo_sk#92] -(94) BroadcastHashJoin [codegen id : 17] +(98) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_promo_sk#77] Right keys [1]: [p_promo_sk#92] Join type: Inner Join condition: None -(95) Project [codegen id : 17] +(99) Project [codegen id : 17] Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] -(96) HashAggregate [codegen id : 17] +(100) HashAggregate [codegen id : 17] Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Keys [1]: [web_site_id#90] Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -(97) Exchange +(101) RowToColumnar Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(98) HashAggregate [codegen id : 18] +(102) CometColumnarExchange +Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(103) ColumnarToRow [codegen id : 18] +Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] + +(104) HashAggregate [codegen id : 18] Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] Keys [1]: [web_site_id#90] Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] Results [5]: [web channel AS channel#106, concat(web_site, web_site_id#90) AS id#107, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#108, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#109, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#110] -(99) Union +(105) Union -(100) HashAggregate [codegen id : 19] +(106) HashAggregate [codegen id : 19] Input [5]: [channel#34, id#35, sales#36, returns#37, profit#38] Keys [2]: [channel#34, id#35] Functions [3]: [partial_sum(sales#36), partial_sum(returns#37), partial_sum(profit#38)] Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] Results [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -(101) Exchange +(107) RowToColumnar +Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] + +(108) CometColumnarExchange Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(102) HashAggregate [codegen id : 20] +(109) CometHashAggregate Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] -Results [5]: [channel#34, id#35, cast(sum(sales#36)#123 as decimal(37,2)) AS sales#126, cast(sum(returns#37)#124 as decimal(38,2)) AS returns#127, cast(sum(profit#38)#125 as decimal(38,2)) AS profit#128] -(103) ReusedExchange [Reuses operator id: 101] +(110) ReusedExchange [Reuses operator id: 108] Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -(104) HashAggregate [codegen id : 40] +(111) CometHashAggregate Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] -Results [4]: [channel#34, sum(sales#36)#123 AS sales#129, sum(returns#37)#124 AS returns#130, sum(profit#38)#125 AS profit#131] -(105) HashAggregate [codegen id : 40] -Input [4]: [channel#34, sales#129, returns#130, profit#131] +(112) CometHashAggregate +Input [4]: [channel#34, sales#123, returns#124, profit#125] Keys [1]: [channel#34] -Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] -Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -Results [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Functions [3]: [partial_sum(sales#123), partial_sum(returns#124), partial_sum(profit#125)] -(106) Exchange -Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [plan_id=16] +(113) CometColumnarExchange +Input [7]: [channel#34, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(107) HashAggregate [codegen id : 41] -Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +(114) CometHashAggregate +Input [7]: [channel#34, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] Keys [1]: [channel#34] -Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] -Aggregate Attributes [3]: [sum(sales#129)#144, sum(returns#130)#145, sum(profit#131)#146] -Results [5]: [channel#34, null AS id#147, sum(sales#129)#144 AS sales#148, sum(returns#130)#145 AS returns#149, sum(profit#131)#146 AS profit#150] +Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] -(108) ReusedExchange [Reuses operator id: 101] +(115) ReusedExchange [Reuses operator id: 108] Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -(109) HashAggregate [codegen id : 61] +(116) CometHashAggregate Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] -Results [3]: [sum(sales#36)#123 AS sales#129, sum(returns#37)#124 AS returns#130, sum(profit#38)#125 AS profit#131] -(110) HashAggregate [codegen id : 61] -Input [3]: [sales#129, returns#130, profit#131] +(117) CometHashAggregate +Input [3]: [sales#123, returns#124, profit#125] Keys: [] -Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] -Aggregate Attributes [6]: [sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] -Results [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +Functions [3]: [partial_sum(sales#123), partial_sum(returns#124), partial_sum(profit#125)] -(111) Exchange -Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] +(118) CometColumnarExchange +Input [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] -(112) HashAggregate [codegen id : 62] -Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +(119) CometHashAggregate +Input [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] Keys: [] -Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] -Aggregate Attributes [3]: [sum(sales#129)#163, sum(returns#130)#164, sum(profit#131)#165] -Results [5]: [null AS channel#166, null AS id#167, sum(sales#129)#163 AS sales#168, sum(returns#130)#164 AS returns#169, sum(profit#131)#165 AS profit#170] +Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] -(113) Union +(120) CometUnion +Child 0 Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] +Child 1 Input [5]: [channel#34, id#141, sales#142, returns#143, profit#144] +Child 2 Input [5]: [channel#145, id#146, sales#147, returns#148, profit#149] -(114) HashAggregate [codegen id : 63] -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +(121) CometHashAggregate +Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] +Keys [5]: [channel#34, id#35, sales#138, returns#139, profit#140] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -(115) Exchange -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Arguments: hashpartitioning(channel#34, id#35, sales#126, returns#127, profit#128, 5), ENSURE_REQUIREMENTS, [plan_id=18] +(122) CometColumnarExchange +Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] +Arguments: hashpartitioning(channel#34, id#35, sales#138, returns#139, profit#140, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(116) HashAggregate [codegen id : 64] -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +(123) CometHashAggregate +Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] +Keys [5]: [channel#34, id#35, sales#138, returns#139, profit#140] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -(117) TakeOrderedAndProject -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#126, returns#127, profit#128] +(124) CometTakeOrderedAndProject +Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#34 ASC NULLS FIRST,id#35 ASC NULLS FIRST], output=[channel#34,id#35,sales#138,returns#139,profit#140]), 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#138, returns#139, profit#140] + +(125) ColumnarToRow [codegen id : 58] +Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (122) -+- * ColumnarToRow (121) - +- CometProject (120) - +- CometFilter (119) - +- CometScan parquet spark_catalog.default.date_dim (118) +BroadcastExchange (130) ++- * ColumnarToRow (129) + +- CometProject (128) + +- CometFilter (127) + +- CometScan parquet spark_catalog.default.date_dim (126) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#171] +Output [2]: [d_date_sk#14, d_date#150] 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 -(119) CometFilter -Input [2]: [d_date_sk#14, d_date#171] -Condition : (((isnotnull(d_date#171) AND (d_date#171 >= 1998-08-04)) AND (d_date#171 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +(127) CometFilter +Input [2]: [d_date_sk#14, d_date#150] +Condition : (((isnotnull(d_date#150) AND (d_date#150 >= 1998-08-04)) AND (d_date#150 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(120) CometProject -Input [2]: [d_date_sk#14, d_date#171] +(128) CometProject +Input [2]: [d_date_sk#14, d_date#150] Arguments: [d_date_sk#14], [d_date_sk#14] -(121) ColumnarToRow [codegen id : 1] +(129) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(122) BroadcastExchange +(130) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 69 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#81 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 f6a37c6372..f694eedfaf 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,24 +1,24 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (64) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (63) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (20) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [channel,id] #2 - WholeStageCodegen (19) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - 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] - InputAdapter - Exchange [s_store_id] #3 +WholeStageCodegen (58) + ColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (19) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + RowToColumnar WholeStageCodegen (5) HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] @@ -34,7 +34,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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_ticket_number,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_ticket_number] - CometExchange [ss_item_sk,ss_ticket_number] #4 + CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] CometScan 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 @@ -46,7 +46,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_item_sk,sr_ticket_number] #6 + CometColumnarExchange [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] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] @@ -75,10 +75,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometProject [p_promo_sk] CometFilter [p_channel_tv,p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (12) - 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] - InputAdapter - Exchange [cp_catalog_page_id] #10 + WholeStageCodegen (12) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #10 + RowToColumnar WholeStageCodegen (11) HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] @@ -94,12 +96,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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_item_sk,cs_order_number,cr_item_sk,cr_order_number] CometSort [cs_item_sk,cs_order_number] - CometExchange [cs_item_sk,cs_order_number] #11 + CometColumnarExchange [cs_item_sk,cs_order_number] #11 CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] CometScan 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] - CometExchange [cr_item_sk,cr_order_number] #12 + CometColumnarExchange [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] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] @@ -116,10 +118,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [i_item_sk] #8 InputAdapter ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (18) - 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] - InputAdapter - Exchange [web_site_id] #14 + WholeStageCodegen (18) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #14 + RowToColumnar WholeStageCodegen (17) HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] @@ -135,12 +139,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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_order_number,wr_item_sk,wr_order_number] CometSort [ws_item_sk,ws_order_number] - CometExchange [ws_item_sk,ws_order_number] #15 + CometColumnarExchange [ws_item_sk,ws_order_number] #15 CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] CometScan 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] - CometExchange [wr_item_sk,wr_order_number] #16 + CometColumnarExchange [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] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] @@ -157,21 +161,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [i_item_sk] #8 InputAdapter ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (41) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange [channel] #18 - WholeStageCodegen (40) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (62) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange #19 - WholeStageCodegen (61) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange [channel] #18 + CometHashAggregate [channel,sales,returns,profit] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarExchange #19 + CometHashAggregate [sales,returns,profit] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt index 610ae89672..2c60857ef7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt @@ -1,38 +1,40 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * Project (33) - +- Window (32) - +- * Sort (31) - +- Exchange (30) - +- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- Union (26) - :- * HashAggregate (15) - : +- Exchange (14) - : +- * HashAggregate (13) - : +- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - :- * HashAggregate (20) - : +- Exchange (19) - : +- * HashAggregate (18) - : +- * HashAggregate (17) - : +- ReusedExchange (16) - +- * HashAggregate (25) - +- Exchange (24) - +- * HashAggregate (23) - +- * HashAggregate (22) - +- ReusedExchange (21) +TakeOrderedAndProject (36) ++- * Project (35) + +- Window (34) + +- * ColumnarToRow (33) + +- CometSort (32) + +- CometColumnarExchange (31) + +- CometHashAggregate (30) + +- CometColumnarExchange (29) + +- CometHashAggregate (28) + +- CometUnion (27) + :- CometHashAggregate (16) + : +- CometColumnarExchange (15) + : +- RowToColumnar (14) + : +- * HashAggregate (13) + : +- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + :- CometHashAggregate (21) + : +- CometColumnarExchange (20) + : +- CometHashAggregate (19) + : +- CometHashAggregate (18) + : +- ReusedExchange (17) + +- CometHashAggregate (26) + +- CometColumnarExchange (25) + +- CometHashAggregate (24) + +- CometHashAggregate (23) + +- ReusedExchange (22) (unknown) Scan parquet spark_catalog.default.web_sales @@ -50,7 +52,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 39] +(4) ReusedExchange [Reuses operator id: 41] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 3] @@ -98,142 +100,133 @@ Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum#9] Results [3]: [i_category#8, i_class#7, sum#10] -(14) Exchange +(14) RowToColumnar Input [3]: [i_category#8, i_class#7, sum#10] -Arguments: hashpartitioning(i_category#8, i_class#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(15) CometColumnarExchange +Input [3]: [i_category#8, i_class#7, sum#10] +Arguments: hashpartitioning(i_category#8, i_class#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [3]: [i_category#8, i_class#7, sum#10] Keys [2]: [i_category#8, i_class#7] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) as decimal(27,2)) AS total_sum#12, i_category#8, i_class#7, 0 AS g_category#13, 0 AS g_class#14, 0 AS lochierarchy#15] -(16) ReusedExchange [Reuses operator id: 14] -Output [3]: [i_category#8, i_class#7, sum#16] +(17) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#8, i_class#7, sum#11] -(17) HashAggregate [codegen id : 8] -Input [3]: [i_category#8, i_class#7, sum#16] +(18) CometHashAggregate +Input [3]: [i_category#8, i_class#7, sum#11] Keys [2]: [i_category#8, i_class#7] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) AS total_sum#17, i_category#8] -(18) HashAggregate [codegen id : 8] -Input [2]: [total_sum#17, i_category#8] +(19) CometHashAggregate +Input [2]: [total_sum#12, i_category#8] Keys [1]: [i_category#8] -Functions [1]: [partial_sum(total_sum#17)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [3]: [i_category#8, sum#20, isEmpty#21] +Functions [1]: [partial_sum(total_sum#12)] -(19) Exchange -Input [3]: [i_category#8, sum#20, isEmpty#21] -Arguments: hashpartitioning(i_category#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(20) CometColumnarExchange +Input [3]: [i_category#8, sum#13, isEmpty#14] +Arguments: hashpartitioning(i_category#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) HashAggregate [codegen id : 9] -Input [3]: [i_category#8, sum#20, isEmpty#21] +(21) CometHashAggregate +Input [3]: [i_category#8, sum#13, isEmpty#14] Keys [1]: [i_category#8] -Functions [1]: [sum(total_sum#17)] -Aggregate Attributes [1]: [sum(total_sum#17)#22] -Results [6]: [sum(total_sum#17)#22 AS total_sum#23, i_category#8, null AS i_class#24, 0 AS g_category#25, 1 AS g_class#26, 1 AS lochierarchy#27] +Functions [1]: [sum(total_sum#12)] -(21) ReusedExchange [Reuses operator id: 14] -Output [3]: [i_category#8, i_class#7, sum#28] +(22) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#8, i_class#7, sum#15] -(22) HashAggregate [codegen id : 13] -Input [3]: [i_category#8, i_class#7, sum#28] +(23) CometHashAggregate +Input [3]: [i_category#8, i_class#7, sum#15] Keys [2]: [i_category#8, i_class#7] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) AS total_sum#17] -(23) HashAggregate [codegen id : 13] -Input [1]: [total_sum#17] +(24) CometHashAggregate +Input [1]: [total_sum#12] Keys: [] -Functions [1]: [partial_sum(total_sum#17)] -Aggregate Attributes [2]: [sum#29, isEmpty#30] -Results [2]: [sum#31, isEmpty#32] +Functions [1]: [partial_sum(total_sum#12)] -(24) Exchange -Input [2]: [sum#31, isEmpty#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(25) CometColumnarExchange +Input [2]: [sum#16, isEmpty#17] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) HashAggregate [codegen id : 14] -Input [2]: [sum#31, isEmpty#32] +(26) CometHashAggregate +Input [2]: [sum#16, isEmpty#17] Keys: [] -Functions [1]: [sum(total_sum#17)] -Aggregate Attributes [1]: [sum(total_sum#17)#33] -Results [6]: [sum(total_sum#17)#33 AS total_sum#34, null AS i_category#35, null AS i_class#36, 1 AS g_category#37, 1 AS g_class#38, 2 AS lochierarchy#39] +Functions [1]: [sum(total_sum#12)] -(26) Union +(27) CometUnion +Child 0 Input [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] +Child 1 Input [6]: [total_sum#22, i_category#8, i_class#23, g_category#24, g_class#25, lochierarchy#26] +Child 2 Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -(27) HashAggregate [codegen id : 15] -Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +(28) CometHashAggregate +Input [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] +Keys [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -(28) Exchange -Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -Arguments: hashpartitioning(total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(29) CometColumnarExchange +Input [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] +Arguments: hashpartitioning(total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(29) HashAggregate [codegen id : 16] -Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +(30) CometHashAggregate +Input [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] +Keys [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] Functions: [] -Aggregate Attributes: [] -Results [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, CASE WHEN (g_class#14 = 0) THEN i_category#8 END AS _w0#40] -(30) Exchange -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] -Arguments: hashpartitioning(lochierarchy#15, _w0#40, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(31) CometColumnarExchange +Input [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33] +Arguments: hashpartitioning(lochierarchy#21, _w0#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(32) CometSort +Input [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33] +Arguments: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33], [lochierarchy#21 ASC NULLS FIRST, _w0#33 ASC NULLS FIRST, total_sum#18 DESC NULLS LAST] -(31) Sort [codegen id : 17] -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] -Arguments: [lochierarchy#15 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#12 DESC NULLS LAST], false, 0 +(33) ColumnarToRow [codegen id : 10] +Input [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33] -(32) Window -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] -Arguments: [rank(total_sum#12) windowspecdefinition(lochierarchy#15, _w0#40, total_sum#12 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [lochierarchy#15, _w0#40], [total_sum#12 DESC NULLS LAST] +(34) Window +Input [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33] +Arguments: [rank(total_sum#18) windowspecdefinition(lochierarchy#21, _w0#33, total_sum#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [lochierarchy#21, _w0#33], [total_sum#18 DESC NULLS LAST] -(33) Project [codegen id : 18] -Output [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] -Input [6]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40, rank_within_parent#41] +(35) Project [codegen id : 11] +Output [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, rank_within_parent#34] +Input [6]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33, rank_within_parent#34] -(34) TakeOrderedAndProject -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] -Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] +(36) TakeOrderedAndProject +Input [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, rank_within_parent#34] +Arguments: 100, [lochierarchy#21 DESC NULLS LAST, CASE WHEN (lochierarchy#21 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#18, i_category#8, i_class#7, lochierarchy#21, rank_within_parent#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (39) -+- * ColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan parquet spark_catalog.default.date_dim (35) +BroadcastExchange (41) ++- * ColumnarToRow (40) + +- CometProject (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.date_dim (37) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#42] +Output [2]: [d_date_sk#5, d_month_seq#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(36) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#42] -Condition : (((isnotnull(d_month_seq#42) AND (d_month_seq#42 >= 1212)) AND (d_month_seq#42 <= 1223)) AND isnotnull(d_date_sk#5)) +(38) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#35] +Condition : (((isnotnull(d_month_seq#35) AND (d_month_seq#35 >= 1212)) AND (d_month_seq#35 <= 1223)) AND isnotnull(d_date_sk#5)) -(37) CometProject -Input [2]: [d_date_sk#5, d_month_seq#42] +(39) CometProject +Input [2]: [d_date_sk#5, d_month_seq#35] Arguments: [d_date_sk#5], [d_date_sk#5] -(38) ColumnarToRow [codegen id : 1] +(40) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(39) BroadcastExchange +(41) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index 5c5e088857..ef041fca19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -1,66 +1,54 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (18) + WholeStageCodegen (11) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (17) - Sort [lochierarchy,_w0,total_sum] + WholeStageCodegen (10) + ColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (16) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] [_w0] - InputAdapter - Exchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (15) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (4) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,g_category,g_class,lochierarchy,sum] - InputAdapter - Exchange [i_category,i_class] #3 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (9) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - InputAdapter - Exchange [i_category] #6 - WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (14) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - InputAdapter - Exchange #7 - WholeStageCodegen (13) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometSort [lochierarchy,_w0,total_sum] + CometColumnarExchange [lochierarchy,_w0] #1 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometUnion + CometHashAggregate [i_category,i_class,sum] + CometColumnarExchange [i_category,i_class] #3 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] + Project [ws_net_paid,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_category,i_class,sum] #3 + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometHashAggregate [i_category,sum,isEmpty] + CometColumnarExchange [i_category] #6 + CometHashAggregate [i_category,total_sum] + CometHashAggregate [i_category,i_class,sum] + ReusedExchange [i_category,i_class,sum] #3 + CometHashAggregate [sum,isEmpty] + CometColumnarExchange #7 + CometHashAggregate [total_sum] + CometHashAggregate [i_category,i_class,sum] + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index 7fa138d5e9..080613d65b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -1,25 +1,29 @@ == Physical Plan == -* Sort (21) -+- Exchange (20) - +- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +* ColumnarToRow (25) ++- CometSort (24) + +- CometColumnarExchange (23) + +- RowToColumnar (22) + +- * Project (21) + +- Window (20) + +- * ColumnarToRow (19) + +- CometSort (18) + +- CometColumnarExchange (17) + +- CometHashAggregate (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.store_sales @@ -65,7 +69,7 @@ Join condition: None Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 26] +(10) ReusedExchange [Reuses operator id: 30] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -85,70 +89,80 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(14) RowToColumnar Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(15) CometColumnarExchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(16) CometHashAggregate Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] -Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16] -(16) Exchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(17) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(18) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] + +(19) ColumnarToRow [codegen id : 4] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] + +(20) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] -(17) Sort [codegen id : 5] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [i_class#9 ASC NULLS FIRST], false, 0 +(21) Project [codegen id : 5] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] -(18) Window -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(22) RowToColumnar +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] -(19) Project [codegen id : 6] -Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] -Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] +(23) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(20) Exchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(24) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST] -(21) Sort [codegen id : 7] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], true, 0 +(25) ColumnarToRow [codegen id : 6] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (26) -+- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +Output [2]: [d_date_sk#11, d_date#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(23) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(27) CometFilter +Input [2]: [d_date_sk#11, d_date#18] +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(24) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(28) CometProject +Input [2]: [d_date_sk#11, d_date#18] Arguments: [d_date_sk#11], [d_date_sk#11] -(25) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(26) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index b7489a0aff..b59521b6c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -1,43 +1,45 @@ -WholeStageCodegen (7) - Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] +WholeStageCodegen (6) + ColumnarToRow InputAdapter - Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - Sort [i_class] - InputAdapter - Exchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - InputAdapter - Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [i_category,i_class,i_item_id,i_item_desc,revenueratio] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + RowToColumnar + WholeStageCodegen (5) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [i_class] + CometColumnarExchange [i_class] #2 + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #4 From 0ab3225ecac17f375393e192f6d923f53707ba45 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 9 Apr 2024 23:45:01 -0700 Subject: [PATCH 03/30] Fix --- .../main/scala/org/apache/comet/vector/NativeUtil.scala | 2 +- .../main/scala/org/apache/spark/sql/comet/util/Utils.scala | 7 ++++--- .../main/scala/org/apache/comet/serde/QueryPlanSerde.scala | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala index eb731f9d0c..595c0a4278 100644 --- a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala +++ b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala @@ -66,7 +66,7 @@ class NativeUtil { val arrowArray = ArrowArray.allocateNew(allocator) Data.exportVector( allocator, - getFieldVector(valueVector), + getFieldVector(valueVector, "export"), provider, arrowArray, arrowSchema) diff --git a/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala b/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala index 7d920e1be1..2300e109aa 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala +++ b/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala @@ -242,7 +242,7 @@ object Utils { } } - getFieldVector(valueVector) + getFieldVector(valueVector, "serialize") case c => throw new SparkException( @@ -253,14 +253,15 @@ object Utils { (fieldVectors, provider) } - def getFieldVector(valueVector: ValueVector): FieldVector = { + def getFieldVector(valueVector: ValueVector, reason: String): FieldVector = { valueVector match { case v @ (_: BitVector | _: TinyIntVector | _: SmallIntVector | _: IntVector | _: BigIntVector | _: Float4Vector | _: Float8Vector | _: VarCharVector | _: DecimalVector | _: DateDayVector | _: TimeStampMicroTZVector | _: VarBinaryVector | _: FixedSizeBinaryVector | _: TimeStampMicroVector) => v.asInstanceOf[FieldVector] - case _ => throw new SparkException(s"Unsupported Arrow Vector: ${valueVector.getClass}") + case _ => + throw new SparkException(s"Unsupported Arrow Vector for $reason: ${valueVector.getClass}") } } } 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 6eda0547f2..c1d787fbb1 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2463,7 +2463,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { withInfo(join, "SortMergeJoin is not enabled") None - case op if isCometSink(op) => + case op if isCometSink(op) && op.output.forall(a => supportedDataType(a.dataType)) => // These operators are source of Comet native execution chain val scanBuilder = OperatorOuterClass.Scan.newBuilder() From b2d9dac36a37d87afb2f12408cc606c7277e5798 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 10 Apr 2024 07:43:09 -0700 Subject: [PATCH 04/30] Update diff --- dev/diffs/3.4.2.diff | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 7c7323d33c..f847756836 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -240,7 +240,7 @@ index 9ddb4abe98b..1bebe99f1cc 100644 sql( """ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..6160c8d241a 100644 +index f33432ddb6f..060f874ea72 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -261,7 +261,17 @@ index f33432ddb6f..6160c8d241a 100644 case _ => Nil } } -@@ -1238,7 +1242,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1187,7 +1191,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("Make sure dynamic pruning works on uncorrelated queries") { ++ test("Make sure dynamic pruning works on uncorrelated queries", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( + """ +@@ -1238,7 +1243,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -271,7 +281,7 @@ index f33432ddb6f..6160c8d241a 100644 Given("dynamic pruning filter on the build side") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( -@@ -1485,7 +1490,7 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1485,7 +1491,7 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-38148: Do not add dynamic partition pruning if there exists static partition " + @@ -280,7 +290,7 @@ index f33432ddb6f..6160c8d241a 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { Seq( "f.store_id = 1" -> false, -@@ -1729,6 +1734,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1735,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) From c43488acd0f4f6a7dea769525ed55904bac7c522 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 11 Apr 2024 13:01:47 -0700 Subject: [PATCH 05/30] Add Comet memoryoverhead for Spark SQL tests --- dev/diffs/3.4.2.diff | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index f847756836..5d3eb790e8 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -1395,10 +1395,10 @@ index dd55fcfe42c..cc18147d17a 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index ed2e309fa07..4cfe0093da7 100644 +index ed2e309fa07..f51a2395a53 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -@@ -74,6 +74,21 @@ trait SharedSparkSessionBase +@@ -74,6 +74,22 @@ trait SharedSparkSessionBase // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) @@ -1415,6 +1415,7 @@ index ed2e309fa07..4cfe0093da7 100644 + .set("spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .set("spark.comet.exec.shuffle.enabled", "true") ++ .set("spark.comet.memoryOverhead", "10g") + } + } conf.set( From 8815eaa2f68d95421d42b0ae5a775acfd9245f2f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 13 Apr 2024 10:56:18 -0700 Subject: [PATCH 06/30] Update plan stability --- .../approved-plans-v1_4/q1/explain.txt | 274 +++-- .../approved-plans-v1_4/q1/simplified.txt | 58 +- .../approved-plans-v1_4/q10/explain.txt | 30 +- .../approved-plans-v1_4/q10/simplified.txt | 10 +- .../approved-plans-v1_4/q11/explain.txt | 298 ++--- .../approved-plans-v1_4/q11/simplified.txt | 28 +- .../approved-plans-v1_4/q12/explain.txt | 116 +- .../approved-plans-v1_4/q12/simplified.txt | 62 +- .../approved-plans-v1_4/q13/explain.txt | 22 +- .../approved-plans-v1_4/q13/simplified.txt | 6 +- .../approved-plans-v1_4/q14a/explain.txt | 717 +++++------ .../approved-plans-v1_4/q14a/simplified.txt | 424 +++---- .../approved-plans-v1_4/q14b/explain.txt | 252 ++-- .../approved-plans-v1_4/q14b/simplified.txt | 94 +- .../approved-plans-v1_4/q15/explain.txt | 30 +- .../approved-plans-v1_4/q15/simplified.txt | 10 +- .../approved-plans-v1_4/q16/explain.txt | 14 +- .../approved-plans-v1_4/q16/simplified.txt | 6 +- .../approved-plans-v1_4/q18/explain.txt | 30 +- .../approved-plans-v1_4/q18/simplified.txt | 10 +- .../approved-plans-v1_4/q19/explain.txt | 22 +- .../approved-plans-v1_4/q19/simplified.txt | 10 +- .../approved-plans-v1_4/q2/explain.txt | 94 +- .../approved-plans-v1_4/q2/simplified.txt | 12 +- .../approved-plans-v1_4/q20/explain.txt | 116 +- .../approved-plans-v1_4/q20/simplified.txt | 62 +- .../approved-plans-v1_4/q21/explain.txt | 30 +- .../approved-plans-v1_4/q21/simplified.txt | 12 +- .../approved-plans-v1_4/q22/explain.txt | 30 +- .../approved-plans-v1_4/q22/simplified.txt | 10 +- .../approved-plans-v1_4/q23a/explain.txt | 716 +++++------ .../approved-plans-v1_4/q23a/simplified.txt | 286 ++--- .../approved-plans-v1_4/q23b/explain.txt | 883 +++++++------- .../approved-plans-v1_4/q23b/simplified.txt | 317 ++--- .../approved-plans-v1_4/q24a/explain.txt | 300 ++--- .../approved-plans-v1_4/q24a/simplified.txt | 158 +-- .../approved-plans-v1_4/q24b/explain.txt | 300 ++--- .../approved-plans-v1_4/q24b/simplified.txt | 158 +-- .../approved-plans-v1_4/q25/explain.txt | 38 +- .../approved-plans-v1_4/q25/simplified.txt | 10 +- .../approved-plans-v1_4/q26/explain.txt | 30 +- .../approved-plans-v1_4/q26/simplified.txt | 10 +- .../approved-plans-v1_4/q27/explain.txt | 30 +- .../approved-plans-v1_4/q27/simplified.txt | 10 +- .../approved-plans-v1_4/q29/explain.txt | 46 +- .../approved-plans-v1_4/q29/simplified.txt | 10 +- .../approved-plans-v1_4/q3/explain.txt | 22 +- .../approved-plans-v1_4/q3/simplified.txt | 10 +- .../approved-plans-v1_4/q30/explain.txt | 280 +++-- .../approved-plans-v1_4/q30/simplified.txt | 66 +- .../approved-plans-v1_4/q31/explain.txt | 416 +++---- .../approved-plans-v1_4/q31/simplified.txt | 36 +- .../approved-plans-v1_4/q32/explain.txt | 72 +- .../approved-plans-v1_4/q32/simplified.txt | 14 +- .../approved-plans-v1_4/q33/explain.txt | 411 ++++--- .../approved-plans-v1_4/q33/simplified.txt | 204 ++-- .../approved-plans-v1_4/q34/explain.txt | 58 +- .../approved-plans-v1_4/q34/simplified.txt | 8 +- .../approved-plans-v1_4/q35/explain.txt | 30 +- .../approved-plans-v1_4/q35/simplified.txt | 10 +- .../approved-plans-v1_4/q36/explain.txt | 134 ++- .../approved-plans-v1_4/q36/simplified.txt | 86 +- .../approved-plans-v1_4/q38/explain.txt | 22 +- .../approved-plans-v1_4/q38/simplified.txt | 6 +- .../approved-plans-v1_4/q4/explain.txt | 472 ++++---- .../approved-plans-v1_4/q4/simplified.txt | 42 +- .../approved-plans-v1_4/q40/explain.txt | 22 +- .../approved-plans-v1_4/q40/simplified.txt | 10 +- .../approved-plans-v1_4/q42/explain.txt | 22 +- .../approved-plans-v1_4/q42/simplified.txt | 10 +- .../approved-plans-v1_4/q43/explain.txt | 22 +- .../approved-plans-v1_4/q43/simplified.txt | 10 +- .../approved-plans-v1_4/q45/explain.txt | 30 +- .../approved-plans-v1_4/q45/simplified.txt | 10 +- .../approved-plans-v1_4/q46/explain.txt | 54 +- .../approved-plans-v1_4/q46/simplified.txt | 6 +- .../approved-plans-v1_4/q47/explain.txt | 281 +++-- .../approved-plans-v1_4/q47/simplified.txt | 100 +- .../approved-plans-v1_4/q48/explain.txt | 22 +- .../approved-plans-v1_4/q48/simplified.txt | 6 +- .../approved-plans-v1_4/q49/explain.txt | 596 ++++----- .../approved-plans-v1_4/q49/simplified.txt | 174 +-- .../approved-plans-v1_4/q5/explain.txt | 30 +- .../approved-plans-v1_4/q5/simplified.txt | 10 +- .../approved-plans-v1_4/q50/explain.txt | 30 +- .../approved-plans-v1_4/q50/simplified.txt | 10 +- .../approved-plans-v1_4/q51/explain.txt | 332 +++--- .../approved-plans-v1_4/q51/simplified.txt | 90 +- .../approved-plans-v1_4/q52/explain.txt | 22 +- .../approved-plans-v1_4/q52/simplified.txt | 10 +- .../approved-plans-v1_4/q53/explain.txt | 138 ++- .../approved-plans-v1_4/q53/simplified.txt | 82 +- .../approved-plans-v1_4/q54/explain.txt | 324 ++--- .../approved-plans-v1_4/q54/simplified.txt | 226 ++-- .../approved-plans-v1_4/q55/explain.txt | 22 +- .../approved-plans-v1_4/q55/simplified.txt | 10 +- .../approved-plans-v1_4/q56/explain.txt | 411 ++++--- .../approved-plans-v1_4/q56/simplified.txt | 204 ++-- .../approved-plans-v1_4/q57/explain.txt | 281 +++-- .../approved-plans-v1_4/q57/simplified.txt | 100 +- .../approved-plans-v1_4/q58/explain.txt | 238 ++-- .../approved-plans-v1_4/q58/simplified.txt | 24 +- .../approved-plans-v1_4/q59/explain.txt | 118 +- .../approved-plans-v1_4/q59/simplified.txt | 12 +- .../approved-plans-v1_4/q6/explain.txt | 64 +- .../approved-plans-v1_4/q6/simplified.txt | 12 +- .../approved-plans-v1_4/q60/explain.txt | 411 ++++--- .../approved-plans-v1_4/q60/simplified.txt | 204 ++-- .../approved-plans-v1_4/q61/explain.txt | 120 +- .../approved-plans-v1_4/q61/simplified.txt | 12 +- .../approved-plans-v1_4/q62/explain.txt | 22 +- .../approved-plans-v1_4/q62/simplified.txt | 10 +- .../approved-plans-v1_4/q63/explain.txt | 138 ++- .../approved-plans-v1_4/q63/simplified.txt | 82 +- .../approved-plans-v1_4/q65/explain.txt | 250 ++-- .../approved-plans-v1_4/q65/simplified.txt | 54 +- .../approved-plans-v1_4/q66/explain.txt | 294 ++--- .../approved-plans-v1_4/q66/simplified.txt | 165 +-- .../approved-plans-v1_4/q67/explain.txt | 130 +- .../approved-plans-v1_4/q67/simplified.txt | 84 +- .../approved-plans-v1_4/q68/explain.txt | 54 +- .../approved-plans-v1_4/q68/simplified.txt | 6 +- .../approved-plans-v1_4/q69/explain.txt | 30 +- .../approved-plans-v1_4/q69/simplified.txt | 10 +- .../approved-plans-v1_4/q7/explain.txt | 30 +- .../approved-plans-v1_4/q7/simplified.txt | 10 +- .../approved-plans-v1_4/q70/explain.txt | 214 ++-- .../approved-plans-v1_4/q70/simplified.txt | 132 +- .../approved-plans-v1_4/q71/explain.txt | 140 ++- .../approved-plans-v1_4/q71/simplified.txt | 128 +- .../approved-plans-v1_4/q73/explain.txt | 58 +- .../approved-plans-v1_4/q73/simplified.txt | 8 +- .../approved-plans-v1_4/q74/explain.txt | 294 ++--- .../approved-plans-v1_4/q74/simplified.txt | 28 +- .../approved-plans-v1_4/q76/explain.txt | 22 +- .../approved-plans-v1_4/q76/simplified.txt | 10 +- .../approved-plans-v1_4/q77/explain.txt | 398 ++++--- .../approved-plans-v1_4/q77/simplified.txt | 46 +- .../approved-plans-v1_4/q78/explain.txt | 514 ++++---- .../approved-plans-v1_4/q78/simplified.txt | 176 +-- .../approved-plans-v1_4/q79/explain.txt | 42 +- .../approved-plans-v1_4/q79/simplified.txt | 6 +- .../approved-plans-v1_4/q8/explain.txt | 74 +- .../approved-plans-v1_4/q8/simplified.txt | 20 +- .../approved-plans-v1_4/q80/explain.txt | 30 +- .../approved-plans-v1_4/q80/simplified.txt | 10 +- .../approved-plans-v1_4/q81/explain.txt | 272 +++-- .../approved-plans-v1_4/q81/simplified.txt | 66 +- .../approved-plans-v1_4/q83/explain.txt | 210 ++-- .../approved-plans-v1_4/q83/simplified.txt | 18 +- .../approved-plans-v1_4/q85/explain.txt | 30 +- .../approved-plans-v1_4/q85/simplified.txt | 10 +- .../approved-plans-v1_4/q86/explain.txt | 120 +- .../approved-plans-v1_4/q86/simplified.txt | 68 +- .../approved-plans-v1_4/q87/explain.txt | 22 +- .../approved-plans-v1_4/q87/simplified.txt | 6 +- .../approved-plans-v1_4/q88/explain.txt | 560 ++++----- .../approved-plans-v1_4/q88/simplified.txt | 48 +- .../approved-plans-v1_4/q89/explain.txt | 136 ++- .../approved-plans-v1_4/q89/simplified.txt | 80 +- .../approved-plans-v1_4/q90/explain.txt | 96 +- .../approved-plans-v1_4/q90/simplified.txt | 12 +- .../approved-plans-v1_4/q91/explain.txt | 146 +-- .../approved-plans-v1_4/q91/simplified.txt | 122 +- .../approved-plans-v1_4/q92/explain.txt | 72 +- .../approved-plans-v1_4/q92/simplified.txt | 14 +- .../approved-plans-v1_4/q93/explain.txt | 22 +- .../approved-plans-v1_4/q93/simplified.txt | 10 +- .../approved-plans-v1_4/q94/explain.txt | 14 +- .../approved-plans-v1_4/q94/simplified.txt | 6 +- .../approved-plans-v1_4/q95/explain.txt | 14 +- .../approved-plans-v1_4/q95/simplified.txt | 6 +- .../approved-plans-v1_4/q96/explain.txt | 14 +- .../approved-plans-v1_4/q96/simplified.txt | 6 +- .../approved-plans-v1_4/q98/explain.txt | 144 +-- .../approved-plans-v1_4/q98/simplified.txt | 64 +- .../approved-plans-v1_4/q99/explain.txt | 22 +- .../approved-plans-v1_4/q99/simplified.txt | 10 +- .../approved-plans-v2_7/q10a/explain.txt | 30 +- .../approved-plans-v2_7/q10a/simplified.txt | 10 +- .../approved-plans-v2_7/q11/explain.txt | 294 ++--- .../approved-plans-v2_7/q11/simplified.txt | 28 +- .../approved-plans-v2_7/q12/explain.txt | 116 +- .../approved-plans-v2_7/q12/simplified.txt | 62 +- .../approved-plans-v2_7/q14/explain.txt | 252 ++-- .../approved-plans-v2_7/q14/simplified.txt | 94 +- .../approved-plans-v2_7/q14a/explain.txt | 1060 +++++++++-------- .../approved-plans-v2_7/q14a/simplified.txt | 504 ++++---- .../approved-plans-v2_7/q18a/explain.txt | 661 +++++----- .../approved-plans-v2_7/q18a/simplified.txt | 48 +- .../approved-plans-v2_7/q20/explain.txt | 116 +- .../approved-plans-v2_7/q20/simplified.txt | 62 +- .../approved-plans-v2_7/q22/explain.txt | 30 +- .../approved-plans-v2_7/q22/simplified.txt | 10 +- .../approved-plans-v2_7/q22a/explain.txt | 313 +++-- .../approved-plans-v2_7/q22a/simplified.txt | 76 +- .../approved-plans-v2_7/q24/explain.txt | 320 ++--- .../approved-plans-v2_7/q24/simplified.txt | 208 ++-- .../approved-plans-v2_7/q27a/explain.txt | 325 ++--- .../approved-plans-v2_7/q27a/simplified.txt | 30 +- .../approved-plans-v2_7/q34/explain.txt | 58 +- .../approved-plans-v2_7/q34/simplified.txt | 8 +- .../approved-plans-v2_7/q35/explain.txt | 30 +- .../approved-plans-v2_7/q35/simplified.txt | 10 +- .../approved-plans-v2_7/q35a/explain.txt | 30 +- .../approved-plans-v2_7/q35a/simplified.txt | 10 +- .../approved-plans-v2_7/q36a/explain.txt | 269 +++-- .../approved-plans-v2_7/q36a/simplified.txt | 120 +- .../approved-plans-v2_7/q47/explain.txt | 281 +++-- .../approved-plans-v2_7/q47/simplified.txt | 100 +- .../approved-plans-v2_7/q49/explain.txt | 596 ++++----- .../approved-plans-v2_7/q49/simplified.txt | 174 +-- .../approved-plans-v2_7/q51a/explain.txt | 644 +++++----- .../approved-plans-v2_7/q51a/simplified.txt | 176 +-- .../approved-plans-v2_7/q57/explain.txt | 281 +++-- .../approved-plans-v2_7/q57/simplified.txt | 100 +- .../approved-plans-v2_7/q5a/explain.txt | 645 +++++----- .../approved-plans-v2_7/q5a/simplified.txt | 438 +++---- .../approved-plans-v2_7/q6/explain.txt | 64 +- .../approved-plans-v2_7/q6/simplified.txt | 12 +- .../approved-plans-v2_7/q67a/explain.txt | 561 +++++---- .../approved-plans-v2_7/q67a/simplified.txt | 218 ++-- .../approved-plans-v2_7/q70a/explain.txt | 339 +++--- .../approved-plans-v2_7/q70a/simplified.txt | 174 +-- .../approved-plans-v2_7/q74/explain.txt | 294 ++--- .../approved-plans-v2_7/q74/simplified.txt | 28 +- .../approved-plans-v2_7/q77a/explain.txt | 767 ++++++------ .../approved-plans-v2_7/q77a/simplified.txt | 338 +++--- .../approved-plans-v2_7/q78/explain.txt | 514 ++++---- .../approved-plans-v2_7/q78/simplified.txt | 176 +-- .../approved-plans-v2_7/q80a/explain.txt | 405 ++++--- .../approved-plans-v2_7/q80a/simplified.txt | 346 +++--- .../approved-plans-v2_7/q86a/explain.txt | 255 ++-- .../approved-plans-v2_7/q86a/simplified.txt | 104 +- .../approved-plans-v2_7/q98/explain.txt | 136 ++- .../approved-plans-v2_7/q98/simplified.txt | 64 +- 236 files changed, 17665 insertions(+), 15420 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt index acab82fdec..e1651dccbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * ColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometHashAggregate (10) +TakeOrderedAndProject (46) ++- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (32) + : : +- * BroadcastHashJoin Inner BuildRight (31) + : : :- * Filter (12) + : : : +- * HashAggregate (11) + : : : +- * ColumnarToRow (10) : : : +- CometColumnarExchange (9) : : : +- RowToColumnar (8) : : : +- * HashAggregate (7) @@ -18,31 +18,33 @@ TakeOrderedAndProject (44) : : : : +- CometFilter (2) : : : : +- CometScan parquet spark_catalog.default.store_returns (1) : : : +- ReusedExchange (4) - : : +- BroadcastExchange (28) - : : +- * ColumnarToRow (27) - : : +- CometFilter (26) - : : +- CometHashAggregate (25) - : : +- CometColumnarExchange (24) - : : +- CometHashAggregate (23) - : : +- CometHashAggregate (22) - : : +- CometColumnarExchange (21) - : : +- RowToColumnar (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * ColumnarToRow (15) - : : : +- CometFilter (14) - : : : +- CometScan parquet spark_catalog.default.store_returns (13) - : : +- ReusedExchange (16) - : +- BroadcastExchange (35) - : +- * ColumnarToRow (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometScan parquet spark_catalog.default.store (31) - +- BroadcastExchange (41) - +- * ColumnarToRow (40) - +- CometFilter (39) - +- CometScan parquet spark_catalog.default.customer (38) + : : +- BroadcastExchange (30) + : : +- * Filter (29) + : : +- * HashAggregate (28) + : : +- * ColumnarToRow (27) + : : +- CometColumnarExchange (26) + : : +- RowToColumnar (25) + : : +- * HashAggregate (24) + : : +- * HashAggregate (23) + : : +- * ColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * ColumnarToRow (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store_returns (13) + : : +- ReusedExchange (16) + : +- BroadcastExchange (37) + : +- * ColumnarToRow (36) + : +- CometProject (35) + : +- CometFilter (34) + : +- CometScan parquet spark_catalog.default.store (33) + +- BroadcastExchange (43) + +- * ColumnarToRow (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.customer (40) (unknown) Scan parquet spark_catalog.default.store_returns @@ -60,7 +62,7 @@ Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) (3) ColumnarToRow [codegen id : 2] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -(4) ReusedExchange [Reuses operator id: 49] +(4) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 2] @@ -87,23 +89,25 @@ Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) CometHashAggregate +(10) ColumnarToRow [codegen id : 9] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] + +(11) HashAggregate [codegen id : 9] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] +Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(11) CometFilter -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Condition : isnotnull(ctr_total_return#11) - -(12) ColumnarToRow [codegen id : 8] -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] +(12) Filter [codegen id : 9] +Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] +Condition : isnotnull(ctr_total_return#12) (unknown) Scan parquet spark_catalog.default.store_returns Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#13)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct @@ -114,7 +118,7 @@ Condition : isnotnull(sr_store_sk#2) (15) ColumnarToRow [codegen id : 4] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -(16) ReusedExchange [Reuses operator id: 49] +(16) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#6] (17) BroadcastHashJoin [codegen id : 4] @@ -131,149 +135,161 @@ Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_s Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum#13] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] +Aggregate Attributes [1]: [sum#14] +Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] (20) RowToColumnar -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] (21) CometColumnarExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] +(22) ColumnarToRow [codegen id : 5] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] + +(23) HashAggregate [codegen id : 5] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] - -(23) CometHashAggregate -Input [2]: [ctr_store_sk#10, ctr_total_return#11] -Keys [1]: [ctr_store_sk#10] -Functions [1]: [partial_avg(ctr_total_return#11)] - -(24) CometColumnarExchange -Input [3]: [ctr_store_sk#10, sum#15, count#16] -Arguments: hashpartitioning(ctr_store_sk#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(25) CometHashAggregate -Input [3]: [ctr_store_sk#10, sum#15, count#16] -Keys [1]: [ctr_store_sk#10] -Functions [1]: [avg(ctr_total_return#11)] - -(26) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#17, ctr_store_sk#10#18] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#17) - -(27) ColumnarToRow [codegen id : 5] -Input [2]: [(avg(ctr_total_return) * 1.2)#17, ctr_store_sk#10#18] - -(28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#17, ctr_store_sk#10#18] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] +Results [2]: [sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] + +(24) HashAggregate [codegen id : 5] +Input [2]: [ctr_store_sk#11, ctr_total_return#12] +Keys [1]: [ctr_store_sk#11] +Functions [1]: [partial_avg(ctr_total_return#12)] +Aggregate Attributes [2]: [sum#16, count#17] +Results [3]: [ctr_store_sk#11, sum#18, count#19] + +(25) RowToColumnar +Input [3]: [ctr_store_sk#11, sum#18, count#19] + +(26) CometColumnarExchange +Input [3]: [ctr_store_sk#11, sum#18, count#19] +Arguments: hashpartitioning(ctr_store_sk#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(27) ColumnarToRow [codegen id : 6] +Input [3]: [ctr_store_sk#11, sum#18, count#19] + +(28) HashAggregate [codegen id : 6] +Input [3]: [ctr_store_sk#11, sum#18, count#19] +Keys [1]: [ctr_store_sk#11] +Functions [1]: [avg(ctr_total_return#12)] +Aggregate Attributes [1]: [avg(ctr_total_return#12)#20] +Results [2]: [(avg(ctr_total_return#12)#20 * 1.2) AS (avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11 AS ctr_store_sk#11#22] + +(29) Filter [codegen id : 6] +Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#21) + +(30) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ctr_store_sk#10] -Right keys [1]: [ctr_store_sk#10#18] +(31) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ctr_store_sk#11] +Right keys [1]: [ctr_store_sk#11#22] Join type: Inner -Join condition: (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#17) +Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#21) -(30) Project [codegen id : 8] -Output [2]: [ctr_customer_sk#9, ctr_store_sk#10] -Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#17, ctr_store_sk#10#18] +(32) Project [codegen id : 9] +Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] +Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] (unknown) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#19, s_state#20] +Output [2]: [s_store_sk#23, s_state#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(32) CometFilter -Input [2]: [s_store_sk#19, s_state#20] -Condition : ((isnotnull(s_state#20) AND (s_state#20 = TN)) AND isnotnull(s_store_sk#19)) +(34) CometFilter +Input [2]: [s_store_sk#23, s_state#24] +Condition : ((isnotnull(s_state#24) AND (s_state#24 = TN)) AND isnotnull(s_store_sk#23)) -(33) CometProject -Input [2]: [s_store_sk#19, s_state#20] -Arguments: [s_store_sk#19], [s_store_sk#19] +(35) CometProject +Input [2]: [s_store_sk#23, s_state#24] +Arguments: [s_store_sk#23], [s_store_sk#23] -(34) ColumnarToRow [codegen id : 6] -Input [1]: [s_store_sk#19] +(36) ColumnarToRow [codegen id : 7] +Input [1]: [s_store_sk#23] -(35) BroadcastExchange -Input [1]: [s_store_sk#19] +(37) BroadcastExchange +Input [1]: [s_store_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ctr_store_sk#10] -Right keys [1]: [s_store_sk#19] +(38) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ctr_store_sk#11] +Right keys [1]: [s_store_sk#23] Join type: Inner Join condition: None -(37) Project [codegen id : 8] -Output [1]: [ctr_customer_sk#9] -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#19] +(39) Project [codegen id : 9] +Output [1]: [ctr_customer_sk#10] +Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#23] (unknown) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#21, c_customer_id#22] +Output [2]: [c_customer_sk#25, c_customer_id#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(39) CometFilter -Input [2]: [c_customer_sk#21, c_customer_id#22] -Condition : isnotnull(c_customer_sk#21) +(41) CometFilter +Input [2]: [c_customer_sk#25, c_customer_id#26] +Condition : isnotnull(c_customer_sk#25) -(40) ColumnarToRow [codegen id : 7] -Input [2]: [c_customer_sk#21, c_customer_id#22] +(42) ColumnarToRow [codegen id : 8] +Input [2]: [c_customer_sk#25, c_customer_id#26] -(41) BroadcastExchange -Input [2]: [c_customer_sk#21, c_customer_id#22] +(43) BroadcastExchange +Input [2]: [c_customer_sk#25, c_customer_id#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ctr_customer_sk#9] -Right keys [1]: [c_customer_sk#21] +(44) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ctr_customer_sk#10] +Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(43) Project [codegen id : 8] -Output [1]: [c_customer_id#22] -Input [3]: [ctr_customer_sk#9, c_customer_sk#21, c_customer_id#22] +(45) Project [codegen id : 9] +Output [1]: [c_customer_id#26] +Input [3]: [ctr_customer_sk#10, c_customer_sk#25, c_customer_id#26] -(44) TakeOrderedAndProject -Input [1]: [c_customer_id#22] -Arguments: 100, [c_customer_id#22 ASC NULLS FIRST], [c_customer_id#22] +(46) TakeOrderedAndProject +Input [1]: [c_customer_id#26] +Arguments: 100, [c_customer_id#26 ASC NULLS FIRST], [c_customer_id#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#23] +Output [2]: [d_date_sk#6, d_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [2]: [d_date_sk#6, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2000)) AND isnotnull(d_date_sk#6)) +(48) CometFilter +Input [2]: [d_date_sk#6, d_year#27] +Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2000)) AND isnotnull(d_date_sk#6)) -(47) CometProject -Input [2]: [d_date_sk#6, d_year#23] +(49) CometProject +Input [2]: [d_date_sk#6, d_year#27] Arguments: [d_date_sk#6], [d_date_sk#6] -(48) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(49) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index c8f2ec9eb9..5bf8a8e46d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -1,15 +1,15 @@ TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (8) + WholeStageCodegen (9) Project [c_customer_id] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk] BroadcastHashJoin [ctr_store_sk,s_store_sk] Project [ctr_customer_sk,ctr_store_sk] BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] - ColumnarToRow - InputAdapter - CometFilter [ctr_total_return] - CometHashAggregate [sr_customer_sk,sr_store_sk,sum] + Filter [ctr_total_return] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 RowToColumnar WholeStageCodegen (2) @@ -32,30 +32,34 @@ TakeOrderedAndProject [c_customer_id] ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [(avg(ctr_total_return) * 1.2)] - CometHashAggregate [ctr_store_sk,sum,count] + WholeStageCodegen (6) + Filter [(avg(ctr_total_return) * 1.2)] + HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_store_sk,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ctr_store_sk] #4 - CometHashAggregate [ctr_store_sk,ctr_total_return] - CometHashAggregate [sr_customer_sk,sr_store_sk,sum] - CometColumnarExchange [sr_customer_sk,sr_store_sk] #5 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [sr_store_sk] - CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [sr_customer_sk,sr_store_sk] #5 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_store_sk] + CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) + WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [s_store_sk] @@ -63,7 +67,7 @@ TakeOrderedAndProject [c_customer_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) + WholeStageCodegen (8) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index a99a75e9a6..dda5b40fac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (45) -+- CometTakeOrderedAndProject (44) - +- CometHashAggregate (43) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- * ColumnarToRow (43) +- CometColumnarExchange (42) +- RowToColumnar (41) +- * HashAggregate (40) @@ -241,17 +241,19 @@ Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purch Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometHashAggregate +(43) ColumnarToRow [codegen id : 10] +Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] + +(44) HashAggregate [codegen id : 10] Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#31] +Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] -(44) CometTakeOrderedAndProject -Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#21 ASC NULLS FIRST,cd_marital_status#22 ASC NULLS FIRST,cd_education_status#23 ASC NULLS FIRST,cd_purchase_estimate#24 ASC NULLS FIRST,cd_credit_rating#25 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[cd_gender#21,cd_marital_status#22,cd_education_status#23,cnt1#31,cd_purchase_estimate#24,cnt2#32,cd_credit_rating#25,cnt3#33,cd_dep_count#26,cnt4#34,cd_dep_employed_count#27,cnt5#35,cd_dep_college_count#28,cnt6#36]), 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] - -(45) ColumnarToRow [codegen id : 10] -Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] +(45) TakeOrderedAndProject +Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] +Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] ===== Subqueries ===== @@ -264,18 +266,18 @@ BroadcastExchange (50) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#37, d_moy#38] +Output [3]: [d_date_sk#9, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct (47) CometFilter -Input [3]: [d_date_sk#9, d_year#37, d_moy#38] -Condition : (((((isnotnull(d_year#37) AND isnotnull(d_moy#38)) AND (d_year#37 = 2002)) AND (d_moy#38 >= 1)) AND (d_moy#38 <= 4)) AND isnotnull(d_date_sk#9)) +Input [3]: [d_date_sk#9, d_year#38, d_moy#39] +Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 1)) AND (d_moy#39 <= 4)) AND isnotnull(d_date_sk#9)) (48) CometProject -Input [3]: [d_date_sk#9, d_year#37, d_moy#38] +Input [3]: [d_date_sk#9, d_year#38, d_moy#39] Arguments: [d_date_sk#9], [d_date_sk#9] (49) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 8923131021..067c8fb293 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] +TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] + WholeStageCodegen (10) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] + ColumnarToRow + InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 RowToColumnar WholeStageCodegen (9) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index a24e545fbc..bccf5e463d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt @@ -6,9 +6,9 @@ TakeOrderedAndProject (80) : +- * BroadcastHashJoin Inner BuildRight (58) : :- * Project (38) : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometHashAggregate (16) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * ColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- RowToColumnar (14) : : : +- * HashAggregate (13) @@ -25,8 +25,8 @@ TakeOrderedAndProject (80) : : : : +- CometScan parquet spark_catalog.default.store_sales (4) : : : +- ReusedExchange (10) : : +- BroadcastExchange (36) - : : +- * ColumnarToRow (35) - : : +- CometHashAggregate (34) + : : +- * HashAggregate (35) + : : +- * ColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- RowToColumnar (32) : : +- * HashAggregate (31) @@ -43,9 +43,9 @@ TakeOrderedAndProject (80) : : : +- CometScan parquet spark_catalog.default.store_sales (22) : : +- ReusedExchange (28) : +- BroadcastExchange (57) - : +- * ColumnarToRow (56) - : +- CometFilter (55) - : +- CometHashAggregate (54) + : +- * Filter (56) + : +- * HashAggregate (55) + : +- * ColumnarToRow (54) : +- CometColumnarExchange (53) : +- RowToColumnar (52) : +- * HashAggregate (51) @@ -62,8 +62,8 @@ TakeOrderedAndProject (80) : : +- CometScan parquet spark_catalog.default.web_sales (42) : +- ReusedExchange (48) +- BroadcastExchange (77) - +- * ColumnarToRow (76) - +- CometHashAggregate (75) + +- * HashAggregate (76) + +- * ColumnarToRow (75) +- CometColumnarExchange (74) +- RowToColumnar (73) +- * HashAggregate (72) @@ -151,301 +151,309 @@ Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferr Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 16] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] + +(17) HashAggregate [codegen id : 16] Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] +Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] -(17) CometFilter -Input [2]: [customer_id#18, year_total#19] -Condition : (isnotnull(year_total#19) AND (year_total#19 > 0.00)) - -(18) ColumnarToRow [codegen id : 16] -Input [2]: [customer_id#18, year_total#19] +(18) Filter [codegen id : 16] +Input [2]: [customer_id#19, year_total#20] +Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Output [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (20) CometFilter -Input [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] -Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_customer_id#21)) +Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#33)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (23) CometFilter -Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -Condition : isnotnull(ss_customer_sk#28) +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Condition : isnotnull(ss_customer_sk#29) (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] (25) BroadcastExchange -Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#20] -Right keys [1]: [ss_customer_sk#28] +Left keys [1]: [c_customer_sk#21] +Right keys [1]: [ss_customer_sk#29] Join type: Inner Join condition: None (27) Project [codegen id : 6] -Output [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -Input [12]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] (28) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#33, d_year#34] +Output [2]: [d_date_sk#34, d_year#35] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#31] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [ss_sold_date_sk#32] +Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None (30) Project [codegen id : 6] -Output [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] -Input [12]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31, d_date_sk#33, d_year#34] +Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] +Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] (31) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] -Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] -Aggregate Attributes [1]: [sum#35] -Results [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] +Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] +Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] +Aggregate Attributes [1]: [sum#36] +Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] (32) RowToColumnar -Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] (33) CometColumnarExchange -Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] -Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] +Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometHashAggregate -Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] -Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] +(34) ColumnarToRow [codegen id : 7] +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -(35) ColumnarToRow [codegen id : 7] -Input [3]: [customer_id#37, customer_preferred_cust_flag#38, year_total#39] +(35) HashAggregate [codegen id : 7] +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] +Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] +Results [3]: [c_customer_id#22 AS customer_id#38, c_preferred_cust_flag#25 AS customer_preferred_cust_flag#39, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#40] (36) BroadcastExchange -Input [3]: [customer_id#37, customer_preferred_cust_flag#38, year_total#39] +Input [3]: [customer_id#38, customer_preferred_cust_flag#39, year_total#40] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] (37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#37] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#38] Join type: Inner Join condition: None (38) Project [codegen id : 16] -Output [4]: [customer_id#18, year_total#19, customer_preferred_cust_flag#38, year_total#39] -Input [5]: [customer_id#18, year_total#19, customer_id#37, customer_preferred_cust_flag#38, year_total#39] +Output [4]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40] +Input [5]: [customer_id#19, year_total#20, customer_id#38, customer_preferred_cust_flag#39, year_total#40] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] +Output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (40) CometFilter -Input [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] -Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) +Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_customer_id#42)) (41) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47] +Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Output [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#53)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (43) CometFilter -Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_bill_customer_sk#48) +Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Condition : isnotnull(ws_bill_customer_sk#49) (44) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] (45) BroadcastExchange -Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] (46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#40] -Right keys [1]: [ws_bill_customer_sk#48] +Left keys [1]: [c_customer_sk#41] +Right keys [1]: [ws_bill_customer_sk#49] Join type: Inner Join condition: None (47) Project [codegen id : 10] -Output [10]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] -Input [12]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Input [12]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] (48) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#53, d_year#54] +Output [2]: [d_date_sk#54, d_year#55] (49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#53] +Left keys [1]: [ws_sold_date_sk#52] +Right keys [1]: [d_date_sk#54] Join type: Inner Join condition: None (50) Project [codegen id : 10] -Output [10]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#54] -Input [12]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51, d_date_sk#53, d_year#54] +Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] +Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52, d_date_sk#54, d_year#55] (51) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#54] -Keys [8]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#50 - ws_ext_discount_amt#49)))] -Aggregate Attributes [1]: [sum#55] -Results [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#56] +Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] +Aggregate Attributes [1]: [sum#56] +Results [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] (52) RowToColumnar -Input [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#56] +Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] (53) CometColumnarExchange -Input [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#56] -Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] +Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(54) CometHashAggregate -Input [9]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54, sum#56] -Keys [8]: [c_customer_id#41, c_first_name#42, c_last_name#43, c_preferred_cust_flag#44, c_birth_country#45, c_login#46, c_email_address#47, d_year#54] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#50 - ws_ext_discount_amt#49)))] +(54) ColumnarToRow [codegen id : 11] +Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] -(55) CometFilter -Input [2]: [customer_id#57, year_total#58] -Condition : (isnotnull(year_total#58) AND (year_total#58 > 0.00)) +(55) HashAggregate [codegen id : 11] +Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58] +Results [2]: [c_customer_id#42 AS customer_id#59, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58,18,2) AS year_total#60] -(56) ColumnarToRow [codegen id : 11] -Input [2]: [customer_id#57, year_total#58] +(56) Filter [codegen id : 11] +Input [2]: [customer_id#59, year_total#60] +Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.00)) (57) BroadcastExchange -Input [2]: [customer_id#57, year_total#58] +Input [2]: [customer_id#59, year_total#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] (58) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#57] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#59] Join type: Inner Join condition: None (59) Project [codegen id : 16] -Output [5]: [customer_id#18, year_total#19, customer_preferred_cust_flag#38, year_total#39, year_total#58] -Input [6]: [customer_id#18, year_total#19, customer_preferred_cust_flag#38, year_total#39, customer_id#57, year_total#58] +Output [5]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60] +Input [6]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, customer_id#59, year_total#60] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66] +Output [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) CometFilter -Input [8]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66] -Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) +Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] +Condition : (isnotnull(c_customer_sk#61) AND isnotnull(c_customer_id#62)) (62) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66] +Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +Output [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#70), dynamicpruningexpression(ws_sold_date_sk#70 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) CometFilter -Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Condition : isnotnull(ws_bill_customer_sk#67) +Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Condition : isnotnull(ws_bill_customer_sk#69) (65) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] (66) BroadcastExchange -Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#59] -Right keys [1]: [ws_bill_customer_sk#67] +Left keys [1]: [c_customer_sk#61] +Right keys [1]: [ws_bill_customer_sk#69] Join type: Inner Join condition: None (68) Project [codegen id : 14] -Output [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Input [12]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Input [12]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] (69) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#72, d_year#73] +Output [2]: [d_date_sk#74, d_year#75] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#70] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ws_sold_date_sk#72] +Right keys [1]: [d_date_sk#74] Join type: Inner Join condition: None (71) Project [codegen id : 14] -Output [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#73] -Input [12]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70, d_date_sk#72, d_year#73] +Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] +Input [12]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72, d_date_sk#74, d_year#75] (72) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#73] -Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] -Aggregate Attributes [1]: [sum#74] -Results [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73, sum#75] +Input [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] +Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] +Aggregate Attributes [1]: [sum#76] +Results [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] (73) RowToColumnar -Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73, sum#75] +Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] (74) CometColumnarExchange -Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73, sum#75] -Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] +Arguments: hashpartitioning(c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(75) CometHashAggregate -Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73, sum#75] -Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#73] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] +(75) ColumnarToRow [codegen id : 15] +Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] -(76) ColumnarToRow [codegen id : 15] -Input [2]: [customer_id#76, year_total#77] +(76) HashAggregate [codegen id : 15] +Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] +Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#58] +Results [2]: [c_customer_id#62 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#58,18,2) AS year_total#79] (77) BroadcastExchange -Input [2]: [customer_id#76, year_total#77] +Input [2]: [customer_id#78, year_total#79] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] (78) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#76] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#78] Join type: Inner -Join condition: (CASE WHEN (year_total#58 > 0.00) THEN (year_total#77 / year_total#58) END > CASE WHEN (year_total#19 > 0.00) THEN (year_total#39 / year_total#19) END) +Join condition: (CASE WHEN (year_total#60 > 0.00) THEN (year_total#79 / year_total#60) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END) (79) Project [codegen id : 16] -Output [1]: [customer_preferred_cust_flag#38] -Input [7]: [customer_id#18, year_total#19, customer_preferred_cust_flag#38, year_total#39, year_total#58, customer_id#76, year_total#77] +Output [1]: [customer_preferred_cust_flag#39] +Input [7]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60, customer_id#78, year_total#79] (80) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#38] -Arguments: 100, [customer_preferred_cust_flag#38 ASC NULLS FIRST], [customer_preferred_cust_flag#38] +Input [1]: [customer_preferred_cust_flag#39] +Arguments: 100, [customer_preferred_cust_flag#39 ASC NULLS FIRST], [customer_preferred_cust_flag#39] ===== Subqueries ===== @@ -474,7 +482,7 @@ Input [2]: [d_date_sk#14, d_year#15] Input [2]: [d_date_sk#14, d_year#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 BroadcastExchange (88) +- * ColumnarToRow (87) +- CometFilter (86) @@ -482,25 +490,25 @@ BroadcastExchange (88) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] +Output [2]: [d_date_sk#34, d_year#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (86) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) +Input [2]: [d_date_sk#34, d_year#35] +Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) (87) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#33, d_year#34] +Input [2]: [d_date_sk#34, d_year#35] (88) BroadcastExchange -Input [2]: [d_date_sk#33, d_year#34] +Input [2]: [d_date_sk#34, d_year#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#70 IN dynamicpruning#32 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#33 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index 2810bbd068..b1c026338c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -6,10 +6,10 @@ TakeOrderedAndProject [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id] Project [customer_id,year_total,customer_preferred_cust_flag,year_total] BroadcastHashJoin [customer_id,customer_id] - ColumnarToRow - InputAdapter - CometFilter [year_total] - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 RowToColumnar WholeStageCodegen (3) @@ -41,9 +41,9 @@ TakeOrderedAndProject [customer_preferred_cust_flag] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 RowToColumnar WholeStageCodegen (6) @@ -75,10 +75,10 @@ TakeOrderedAndProject [customer_preferred_cust_flag] InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [year_total] - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 RowToColumnar WholeStageCodegen (10) @@ -104,9 +104,9 @@ TakeOrderedAndProject [customer_preferred_cust_flag] InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 RowToColumnar WholeStageCodegen (14) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index b59f6739bb..5e524033e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -1,26 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * Project (21) - +- Window (20) - +- * ColumnarToRow (19) - +- CometSort (18) - +- CometColumnarExchange (17) - +- CometHashAggregate (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * ColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- RowToColumnar (18) + +- * HashAggregate (17) + +- * ColumnarToRow (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.web_sales @@ -66,7 +68,7 @@ Join condition: None Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 27] +(10) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -93,63 +95,71 @@ Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] +Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] + +(18) RowToColumnar +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -(17) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +(19) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) CometSort -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] -Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] +(20) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 4] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +(21) ColumnarToRow [codegen id : 5] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -(20) Window -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] -Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] +(22) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(21) Project [codegen id : 5] -Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] -Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] +(23) Project [codegen id : 6] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] -(22) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +(24) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#18] +Output [2]: [d_date_sk#11, d_date#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [d_date_sk#11, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(26) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(25) CometProject -Input [2]: [d_date_sk#11, d_date#18] +(27) CometProject +Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index 985a824e6e..f1dcf3ef9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -1,40 +1,44 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (5) + WholeStageCodegen (6) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (4) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometSort [i_class] CometColumnarExchange [i_class] #1 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt index a6d0b9b416..61832c82d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* ColumnarToRow (36) -+- CometHashAggregate (35) +* HashAggregate (36) ++- * ColumnarToRow (35) +- CometColumnarExchange (34) +- RowToColumnar (33) +- * HashAggregate (32) @@ -195,13 +195,15 @@ Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(35) CometHashAggregate +(35) ColumnarToRow [codegen id : 7] +Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] + +(36) HashAggregate [codegen id : 7] Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(36) ColumnarToRow [codegen id : 7] -Input [4]: [avg(ss_quantity)#36, avg(ss_ext_sales_price)#37, avg(ss_ext_wholesale_cost)#38, sum(ss_ext_wholesale_cost)#39] +Aggregate Attributes [4]: [avg(ss_quantity#5)#36, avg(UnscaledValue(ss_ext_sales_price#7))#37, avg(UnscaledValue(ss_ext_wholesale_cost#8))#38, sum(UnscaledValue(ss_ext_wholesale_cost#8))#39] +Results [4]: [avg(ss_quantity#5)#36 AS avg(ss_quantity)#40, cast((avg(UnscaledValue(ss_ext_sales_price#7))#37 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#41, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#38 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#42, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#39,17,2) AS sum(ss_ext_wholesale_cost)#43] ===== Subqueries ===== @@ -214,18 +216,18 @@ BroadcastExchange (41) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_year#40] +Output [2]: [d_date_sk#16, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (38) CometFilter -Input [2]: [d_date_sk#16, d_year#40] -Condition : ((isnotnull(d_year#40) AND (d_year#40 = 2001)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#16, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2001)) AND isnotnull(d_date_sk#16)) (39) CometProject -Input [2]: [d_date_sk#16, d_year#40] +Input [2]: [d_date_sk#16, d_year#44] Arguments: [d_date_sk#16], [d_date_sk#16] (40) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index 87b7bf67e8..02184e3c7a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum] + HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (6) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index bfb15cc6a7..a57cf6fcfc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -1,115 +1,119 @@ == Physical Plan == -* ColumnarToRow (111) -+- CometTakeOrderedAndProject (110) - +- CometHashAggregate (109) - +- CometColumnarExchange (108) - +- CometHashAggregate (107) - +- CometExpand (106) - +- CometUnion (105) - :- CometProject (70) - : +- CometFilter (69) - : +- CometHashAggregate (68) - : +- CometColumnarExchange (67) - : +- RowToColumnar (66) - : +- * HashAggregate (65) - : +- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * BroadcastHashJoin LeftSemi BuildRight (53) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (52) - : : : +- * Project (51) - : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (49) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) - : : : :- * ColumnarToRow (37) - : : : : +- CometHashAggregate (36) - : : : : +- CometColumnarExchange (35) - : : : : +- RowToColumnar (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (47) - : : : +- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * Project (43) - : : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : : :- * ColumnarToRow (40) - : : : : : +- CometFilter (39) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) - : : : : +- ReusedExchange (41) - : : : +- ReusedExchange (44) - : : +- BroadcastExchange (59) - : : +- * BroadcastHashJoin LeftSemi BuildRight (58) - : : :- * ColumnarToRow (56) - : : : +- CometFilter (55) - : : : +- CometScan parquet spark_catalog.default.item (54) - : : +- ReusedExchange (57) - : +- ReusedExchange (62) - :- CometProject (87) - : +- CometFilter (86) - : +- CometHashAggregate (85) - : +- CometColumnarExchange (84) - : +- RowToColumnar (83) - : +- * HashAggregate (82) - : +- * Project (81) - : +- * BroadcastHashJoin Inner BuildRight (80) - : :- * Project (78) - : : +- * BroadcastHashJoin Inner BuildRight (77) - : : :- * BroadcastHashJoin LeftSemi BuildRight (75) - : : : :- * ColumnarToRow (73) - : : : : +- CometFilter (72) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (71) - : : : +- ReusedExchange (74) - : : +- ReusedExchange (76) - : +- ReusedExchange (79) - +- CometProject (104) - +- CometFilter (103) - +- CometHashAggregate (102) - +- CometColumnarExchange (101) - +- RowToColumnar (100) - +- * HashAggregate (99) - +- * Project (98) - +- * BroadcastHashJoin Inner BuildRight (97) - :- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * BroadcastHashJoin LeftSemi BuildRight (92) - : : :- * ColumnarToRow (90) - : : : +- CometFilter (89) - : : : +- CometScan parquet spark_catalog.default.web_sales (88) - : : +- ReusedExchange (91) - : +- ReusedExchange (93) - +- ReusedExchange (96) +TakeOrderedAndProject (115) ++- * HashAggregate (114) + +- * ColumnarToRow (113) + +- CometColumnarExchange (112) + +- RowToColumnar (111) + +- * HashAggregate (110) + +- * Expand (109) + +- Union (108) + :- * Project (71) + : +- * Filter (70) + : +- * HashAggregate (69) + : +- * ColumnarToRow (68) + : +- CometColumnarExchange (67) + : +- RowToColumnar (66) + : +- * HashAggregate (65) + : +- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * BroadcastHashJoin LeftSemi BuildRight (53) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (52) + : : : +- * Project (51) + : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : :- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (49) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) + : : : :- * ColumnarToRow (37) + : : : : +- CometHashAggregate (36) + : : : : +- CometColumnarExchange (35) + : : : : +- RowToColumnar (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * ColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (47) + : : : +- * Project (46) + : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : :- * Project (43) + : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : :- * ColumnarToRow (40) + : : : : : +- CometFilter (39) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) + : : : : +- ReusedExchange (41) + : : : +- ReusedExchange (44) + : : +- BroadcastExchange (59) + : : +- * BroadcastHashJoin LeftSemi BuildRight (58) + : : :- * ColumnarToRow (56) + : : : +- CometFilter (55) + : : : +- CometScan parquet spark_catalog.default.item (54) + : : +- ReusedExchange (57) + : +- ReusedExchange (62) + :- * Project (89) + : +- * Filter (88) + : +- * HashAggregate (87) + : +- * ColumnarToRow (86) + : +- CometColumnarExchange (85) + : +- RowToColumnar (84) + : +- * HashAggregate (83) + : +- * Project (82) + : +- * BroadcastHashJoin Inner BuildRight (81) + : :- * Project (79) + : : +- * BroadcastHashJoin Inner BuildRight (78) + : : :- * BroadcastHashJoin LeftSemi BuildRight (76) + : : : :- * ColumnarToRow (74) + : : : : +- CometFilter (73) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) + : : : +- ReusedExchange (75) + : : +- ReusedExchange (77) + : +- ReusedExchange (80) + +- * Project (107) + +- * Filter (106) + +- * HashAggregate (105) + +- * ColumnarToRow (104) + +- CometColumnarExchange (103) + +- RowToColumnar (102) + +- * HashAggregate (101) + +- * Project (100) + +- * BroadcastHashJoin Inner BuildRight (99) + :- * Project (97) + : +- * BroadcastHashJoin Inner BuildRight (96) + : :- * BroadcastHashJoin LeftSemi BuildRight (94) + : : :- * ColumnarToRow (92) + : : : +- CometFilter (91) + : : : +- CometScan parquet spark_catalog.default.web_sales (90) + : : +- ReusedExchange (93) + : +- ReusedExchange (95) + +- ReusedExchange (98) (unknown) Scan parquet spark_catalog.default.store_sales @@ -213,7 +217,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 142] +(22) ReusedExchange [Reuses operator id: 146] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -250,7 +254,7 @@ Join condition: None Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 142] +(30) ReusedExchange [Reuses operator id: 146] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 6] @@ -313,7 +317,7 @@ Join condition: None Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(44) ReusedExchange [Reuses operator id: 142] +(44) ReusedExchange [Reuses operator id: 146] Output [1]: [d_date_sk#36] (45) BroadcastHashJoin [codegen id : 9] @@ -397,7 +401,7 @@ Join condition: None Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(62) ReusedExchange [Reuses operator id: 137] +(62) ReusedExchange [Reuses operator id: 141] Output [1]: [d_date_sk#42] (63) BroadcastHashJoin [codegen id : 25] @@ -424,385 +428,406 @@ Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) CometHashAggregate +(68) ColumnarToRow [codegen id : 26] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(69) HashAggregate [codegen id : 26] Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#51, count(1)#50 AS number_sales#52] -(69) CometFilter -Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#49, number_sales#50] -Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(70) Filter [codegen id : 26] +Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] +Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) -(70) CometProject -Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#49, number_sales#50] -Arguments: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56], [sales#49, number_sales#50, store AS channel#53, i_brand_id#39 AS i_brand_id#54, i_class_id#40 AS i_class_id#55, i_category_id#41 AS i_category_id#56] +(71) Project [codegen id : 26] +Output [6]: [sales#51, number_sales#52, store AS channel#55, i_brand_id#39 AS i_brand_id#56, i_class_id#40 AS i_class_id#57, i_category_id#41 AS i_category_id#58] +Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +Output [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#62), dynamicpruningexpression(cs_sold_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(72) CometFilter -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Condition : isnotnull(cs_item_sk#57) +(73) CometFilter +Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +Condition : isnotnull(cs_item_sk#59) -(73) ColumnarToRow [codegen id : 50] -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +(74) ColumnarToRow [codegen id : 51] +Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] -(74) ReusedExchange [Reuses operator id: 52] +(75) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(75) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [cs_item_sk#57] +(76) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#59] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(76) ReusedExchange [Reuses operator id: 59] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] +(77) ReusedExchange [Reuses operator id: 59] +Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] -(77) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [cs_item_sk#57] -Right keys [1]: [i_item_sk#62] +(78) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#59] +Right keys [1]: [i_item_sk#64] Join type: Inner Join condition: None -(78) Project [codegen id : 50] -Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] +(79) Project [codegen id : 51] +Output [6]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [8]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] -(79) ReusedExchange [Reuses operator id: 137] -Output [1]: [d_date_sk#66] +(80) ReusedExchange [Reuses operator id: 141] +Output [1]: [d_date_sk#68] -(80) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#66] +(81) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_sold_date_sk#62] +Right keys [1]: [d_date_sk#68] Join type: Inner Join condition: None -(81) Project [codegen id : 50] -Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] +(82) Project [codegen id : 51] +Output [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [7]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] -(82) HashAggregate [codegen id : 50] -Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +(83) HashAggregate [codegen id : 51] +Input [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] +Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +Functions [2]: [partial_sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), partial_count(1)] +Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] +Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -(83) RowToColumnar -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +(84) RowToColumnar +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -(84) CometColumnarExchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(85) CometColumnarExchange +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(85) CometHashAggregate -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] +(86) ColumnarToRow [codegen id : 52] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -(86) CometFilter -Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(87) HashAggregate [codegen id : 52] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +Functions [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#75, count(1)#76] +Results [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#75 AS sales#77, count(1)#76 AS number_sales#78] -(87) CometProject -Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#73, number_sales#74] -Arguments: [sales#73, number_sales#74, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65], [sales#73, number_sales#74, catalog AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65] +(88) Filter [codegen id : 52] +Input [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#77, number_sales#78] +Condition : (isnotnull(sales#77) AND (cast(sales#77 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) + +(89) Project [codegen id : 52] +Output [6]: [sales#77, number_sales#78, catalog AS channel#79, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#77, number_sales#78] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79] +Output [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#80)] +PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(89) CometFilter -Input [4]: [ws_item_sk#76, ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_item_sk#76) +(91) CometFilter +Input [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] +Condition : isnotnull(ws_item_sk#80) -(90) ColumnarToRow [codegen id : 75] -Input [4]: [ws_item_sk#76, ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79] +(92) ColumnarToRow [codegen id : 77] +Input [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] -(91) ReusedExchange [Reuses operator id: 52] +(93) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(92) BroadcastHashJoin [codegen id : 75] -Left keys [1]: [ws_item_sk#76] +(94) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#80] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(93) ReusedExchange [Reuses operator id: 59] -Output [4]: [i_item_sk#81, i_brand_id#82, i_class_id#83, i_category_id#84] +(95) ReusedExchange [Reuses operator id: 59] +Output [4]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] -(94) BroadcastHashJoin [codegen id : 75] -Left keys [1]: [ws_item_sk#76] -Right keys [1]: [i_item_sk#81] +(96) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#80] +Right keys [1]: [i_item_sk#85] Join type: Inner Join condition: None -(95) Project [codegen id : 75] -Output [6]: [ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79, i_brand_id#82, i_class_id#83, i_category_id#84] -Input [8]: [ws_item_sk#76, ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79, i_item_sk#81, i_brand_id#82, i_class_id#83, i_category_id#84] +(97) Project [codegen id : 77] +Output [6]: [ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88] +Input [8]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] -(96) ReusedExchange [Reuses operator id: 137] -Output [1]: [d_date_sk#85] +(98) ReusedExchange [Reuses operator id: 141] +Output [1]: [d_date_sk#89] -(97) BroadcastHashJoin [codegen id : 75] -Left keys [1]: [ws_sold_date_sk#79] -Right keys [1]: [d_date_sk#85] +(99) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_sold_date_sk#83] +Right keys [1]: [d_date_sk#89] Join type: Inner Join condition: None -(98) Project [codegen id : 75] -Output [5]: [ws_quantity#77, ws_list_price#78, i_brand_id#82, i_class_id#83, i_category_id#84] -Input [7]: [ws_quantity#77, ws_list_price#78, ws_sold_date_sk#79, i_brand_id#82, i_class_id#83, i_category_id#84, d_date_sk#85] +(100) Project [codegen id : 77] +Output [5]: [ws_quantity#81, ws_list_price#82, i_brand_id#86, i_class_id#87, i_category_id#88] +Input [7]: [ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, d_date_sk#89] + +(101) HashAggregate [codegen id : 77] +Input [5]: [ws_quantity#81, ws_list_price#82, i_brand_id#86, i_class_id#87, i_category_id#88] +Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] +Functions [2]: [partial_sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82)), partial_count(1)] +Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] +Results [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] -(99) HashAggregate [codegen id : 75] -Input [5]: [ws_quantity#77, ws_list_price#78, i_brand_id#82, i_class_id#83, i_category_id#84] -Keys [3]: [i_brand_id#82, i_class_id#83, i_category_id#84] -Functions [2]: [partial_sum((cast(ws_quantity#77 as decimal(10,0)) * ws_list_price#78)), partial_count(1)] -Aggregate Attributes [3]: [sum#86, isEmpty#87, count#88] -Results [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] +(102) RowToColumnar +Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] -(100) RowToColumnar -Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] +(103) CometColumnarExchange +Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(101) CometColumnarExchange -Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] -Arguments: hashpartitioning(i_brand_id#82, i_class_id#83, i_category_id#84, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(104) ColumnarToRow [codegen id : 78] +Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] -(102) CometHashAggregate -Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] -Keys [3]: [i_brand_id#82, i_class_id#83, i_category_id#84] -Functions [2]: [sum((cast(ws_quantity#77 as decimal(10,0)) * ws_list_price#78)), count(1)] +(105) HashAggregate [codegen id : 78] +Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] +Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] +Functions [2]: [sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82))#96, count(1)#97] +Results [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82))#96 AS sales#98, count(1)#97 AS number_sales#99] -(103) CometFilter -Input [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sales#92, number_sales#93] -Condition : (isnotnull(sales#92) AND (cast(sales#92 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(106) Filter [codegen id : 78] +Input [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sales#98, number_sales#99] +Condition : (isnotnull(sales#98) AND (cast(sales#98 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) -(104) CometProject -Input [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sales#92, number_sales#93] -Arguments: [sales#92, number_sales#93, channel#94, i_brand_id#82, i_class_id#83, i_category_id#84], [sales#92, number_sales#93, web AS channel#94, i_brand_id#82, i_class_id#83, i_category_id#84] +(107) Project [codegen id : 78] +Output [6]: [sales#98, number_sales#99, web AS channel#100, i_brand_id#86, i_class_id#87, i_category_id#88] +Input [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sales#98, number_sales#99] -(105) CometUnion -Child 0 Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Child 1 Input [6]: [sales#73, number_sales#74, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65] -Child 2 Input [6]: [sales#92, number_sales#93, channel#94, i_brand_id#82, i_class_id#83, i_category_id#84] +(108) Union -(106) CometExpand -Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99] +(109) Expand [codegen id : 79] +Input [6]: [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58] +Arguments: [[sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58, 0], [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, null, 1], [sales#51, number_sales#52, channel#55, i_brand_id#56, null, null, 3], [sales#51, number_sales#52, channel#55, null, null, null, 7], [sales#51, number_sales#52, null, null, null, null, 15]], [sales#51, number_sales#52, channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] -(107) CometHashAggregate -Input [7]: [sales#49, number_sales#50, channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99] -Keys [5]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99] -Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] +(110) HashAggregate [codegen id : 79] +Input [7]: [sales#51, number_sales#52, channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] +Keys [5]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] +Functions [2]: [partial_sum(sales#51), partial_sum(number_sales#52)] +Aggregate Attributes [3]: [sum#106, isEmpty#107, sum#108] +Results [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] -(108) CometColumnarExchange -Input [8]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99, sum#100, isEmpty#101, sum#102] -Arguments: hashpartitioning(channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +(111) RowToColumnar +Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] -(109) CometHashAggregate -Input [8]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99, sum#100, isEmpty#101, sum#102] -Keys [5]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, spark_grouping_id#99] -Functions [2]: [sum(sales#49), sum(number_sales#50)] +(112) CometColumnarExchange +Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] +Arguments: hashpartitioning(channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(110) CometTakeOrderedAndProject -Input [6]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, sum(sales)#103, sum(number_sales)#104] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#95 ASC NULLS FIRST,i_brand_id#96 ASC NULLS FIRST,i_class_id#97 ASC NULLS FIRST,i_category_id#98 ASC NULLS FIRST], output=[channel#95,i_brand_id#96,i_class_id#97,i_category_id#98,sum(sales)#103,sum(number_sales)#104]), 100, [channel#95 ASC NULLS FIRST, i_brand_id#96 ASC NULLS FIRST, i_class_id#97 ASC NULLS FIRST, i_category_id#98 ASC NULLS FIRST], [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, sum(sales)#103, sum(number_sales)#104] +(113) ColumnarToRow [codegen id : 80] +Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] -(111) ColumnarToRow [codegen id : 76] -Input [6]: [channel#95, i_brand_id#96, i_class_id#97, i_category_id#98, sum(sales)#103, sum(number_sales)#104] +(114) HashAggregate [codegen id : 80] +Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] +Keys [5]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] +Functions [2]: [sum(sales#51), sum(number_sales#52)] +Aggregate Attributes [2]: [sum(sales#51)#112, sum(number_sales#52)#113] +Results [6]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales#51)#112 AS sum(sales)#114, sum(number_sales#52)#113 AS sum(number_sales)#115] + +(115) TakeOrderedAndProject +Input [6]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales)#114, sum(number_sales)#115] +Arguments: 100, [channel#101 ASC NULLS FIRST, i_brand_id#102 ASC NULLS FIRST, i_class_id#103 ASC NULLS FIRST, i_category_id#104 ASC NULLS FIRST], [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales)#114, sum(number_sales)#115] ===== Subqueries ===== -Subquery:1 Hosting operator id = 69 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* ColumnarToRow (132) -+- CometHashAggregate (131) - +- CometColumnarExchange (130) - +- RowToColumnar (129) - +- * HashAggregate (128) - +- Union (127) - :- * Project (116) - : +- * BroadcastHashJoin Inner BuildRight (115) - : :- * ColumnarToRow (113) - : : +- CometScan parquet spark_catalog.default.store_sales (112) - : +- ReusedExchange (114) - :- * Project (121) - : +- * BroadcastHashJoin Inner BuildRight (120) - : :- * ColumnarToRow (118) - : : +- CometScan parquet spark_catalog.default.catalog_sales (117) - : +- ReusedExchange (119) - +- * Project (126) - +- * BroadcastHashJoin Inner BuildRight (125) - :- * ColumnarToRow (123) - : +- CometScan parquet spark_catalog.default.web_sales (122) - +- ReusedExchange (124) +Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#53, [id=#54] +* HashAggregate (136) ++- * ColumnarToRow (135) + +- CometColumnarExchange (134) + +- RowToColumnar (133) + +- * HashAggregate (132) + +- Union (131) + :- * Project (120) + : +- * BroadcastHashJoin Inner BuildRight (119) + : :- * ColumnarToRow (117) + : : +- CometScan parquet spark_catalog.default.store_sales (116) + : +- ReusedExchange (118) + :- * Project (125) + : +- * BroadcastHashJoin Inner BuildRight (124) + : :- * ColumnarToRow (122) + : : +- CometScan parquet spark_catalog.default.catalog_sales (121) + : +- ReusedExchange (123) + +- * Project (130) + +- * BroadcastHashJoin Inner BuildRight (129) + :- * ColumnarToRow (127) + : +- CometScan parquet spark_catalog.default.web_sales (126) + +- ReusedExchange (128) (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#105, ss_list_price#106, ss_sold_date_sk#107] +Output [3]: [ss_quantity#116, ss_list_price#117, ss_sold_date_sk#118] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#107), dynamicpruningexpression(ss_sold_date_sk#107 IN dynamicpruning#108)] +PartitionFilters: [isnotnull(ss_sold_date_sk#118), dynamicpruningexpression(ss_sold_date_sk#118 IN dynamicpruning#119)] ReadSchema: struct -(113) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#105, ss_list_price#106, ss_sold_date_sk#107] +(117) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#116, ss_list_price#117, ss_sold_date_sk#118] -(114) ReusedExchange [Reuses operator id: 142] -Output [1]: [d_date_sk#109] +(118) ReusedExchange [Reuses operator id: 146] +Output [1]: [d_date_sk#120] -(115) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#107] -Right keys [1]: [d_date_sk#109] +(119) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#118] +Right keys [1]: [d_date_sk#120] Join type: Inner Join condition: None -(116) Project [codegen id : 2] -Output [2]: [ss_quantity#105 AS quantity#110, ss_list_price#106 AS list_price#111] -Input [4]: [ss_quantity#105, ss_list_price#106, ss_sold_date_sk#107, d_date_sk#109] +(120) Project [codegen id : 2] +Output [2]: [ss_quantity#116 AS quantity#121, ss_list_price#117 AS list_price#122] +Input [4]: [ss_quantity#116, ss_list_price#117, ss_sold_date_sk#118, d_date_sk#120] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#112, cs_list_price#113, cs_sold_date_sk#114] +Output [3]: [cs_quantity#123, cs_list_price#124, cs_sold_date_sk#125] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#114), dynamicpruningexpression(cs_sold_date_sk#114 IN dynamicpruning#115)] +PartitionFilters: [isnotnull(cs_sold_date_sk#125), dynamicpruningexpression(cs_sold_date_sk#125 IN dynamicpruning#126)] ReadSchema: struct -(118) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#112, cs_list_price#113, cs_sold_date_sk#114] +(122) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#123, cs_list_price#124, cs_sold_date_sk#125] -(119) ReusedExchange [Reuses operator id: 142] -Output [1]: [d_date_sk#116] +(123) ReusedExchange [Reuses operator id: 146] +Output [1]: [d_date_sk#127] -(120) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#114] -Right keys [1]: [d_date_sk#116] +(124) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#125] +Right keys [1]: [d_date_sk#127] Join type: Inner Join condition: None -(121) Project [codegen id : 4] -Output [2]: [cs_quantity#112 AS quantity#117, cs_list_price#113 AS list_price#118] -Input [4]: [cs_quantity#112, cs_list_price#113, cs_sold_date_sk#114, d_date_sk#116] +(125) Project [codegen id : 4] +Output [2]: [cs_quantity#123 AS quantity#128, cs_list_price#124 AS list_price#129] +Input [4]: [cs_quantity#123, cs_list_price#124, cs_sold_date_sk#125, d_date_sk#127] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +Output [3]: [ws_quantity#130, ws_list_price#131, ws_sold_date_sk#132] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#122)] +PartitionFilters: [isnotnull(ws_sold_date_sk#132), dynamicpruningexpression(ws_sold_date_sk#132 IN dynamicpruning#133)] ReadSchema: struct -(123) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +(127) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#130, ws_list_price#131, ws_sold_date_sk#132] -(124) ReusedExchange [Reuses operator id: 142] -Output [1]: [d_date_sk#123] +(128) ReusedExchange [Reuses operator id: 146] +Output [1]: [d_date_sk#134] -(125) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#121] -Right keys [1]: [d_date_sk#123] +(129) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#132] +Right keys [1]: [d_date_sk#134] Join type: Inner Join condition: None -(126) Project [codegen id : 6] -Output [2]: [ws_quantity#119 AS quantity#124, ws_list_price#120 AS list_price#125] -Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#123] +(130) Project [codegen id : 6] +Output [2]: [ws_quantity#130 AS quantity#135, ws_list_price#131 AS list_price#136] +Input [4]: [ws_quantity#130, ws_list_price#131, ws_sold_date_sk#132, d_date_sk#134] -(127) Union +(131) Union -(128) HashAggregate [codegen id : 7] -Input [2]: [quantity#110, list_price#111] +(132) HashAggregate [codegen id : 7] +Input [2]: [quantity#121, list_price#122] Keys: [] -Functions [1]: [partial_avg((cast(quantity#110 as decimal(10,0)) * list_price#111))] -Aggregate Attributes [2]: [sum#126, count#127] -Results [2]: [sum#128, count#129] +Functions [1]: [partial_avg((cast(quantity#121 as decimal(10,0)) * list_price#122))] +Aggregate Attributes [2]: [sum#137, count#138] +Results [2]: [sum#139, count#140] -(129) RowToColumnar -Input [2]: [sum#128, count#129] +(133) RowToColumnar +Input [2]: [sum#139, count#140] -(130) CometColumnarExchange -Input [2]: [sum#128, count#129] +(134) CometColumnarExchange +Input [2]: [sum#139, count#140] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(131) CometHashAggregate -Input [2]: [sum#128, count#129] -Keys: [] -Functions [1]: [avg((cast(quantity#110 as decimal(10,0)) * list_price#111))] +(135) ColumnarToRow [codegen id : 8] +Input [2]: [sum#139, count#140] -(132) ColumnarToRow [codegen id : 8] -Input [1]: [average_sales#130] +(136) HashAggregate [codegen id : 8] +Input [2]: [sum#139, count#140] +Keys: [] +Functions [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))] +Aggregate Attributes [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))#141] +Results [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))#141 AS average_sales#142] -Subquery:2 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#107 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#118 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 117 Hosting Expression = cs_sold_date_sk#114 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 121 Hosting Expression = cs_sold_date_sk#125 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 122 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 126 Hosting Expression = ws_sold_date_sk#132 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (137) -+- * ColumnarToRow (136) - +- CometProject (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +BroadcastExchange (141) ++- * ColumnarToRow (140) + +- CometProject (139) + +- CometFilter (138) + +- CometScan parquet spark_catalog.default.date_dim (137) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#42, d_year#131, d_moy#132] +Output [3]: [d_date_sk#42, d_year#143, d_moy#144] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(134) CometFilter -Input [3]: [d_date_sk#42, d_year#131, d_moy#132] -Condition : ((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND (d_year#131 = 2001)) AND (d_moy#132 = 11)) AND isnotnull(d_date_sk#42)) +(138) CometFilter +Input [3]: [d_date_sk#42, d_year#143, d_moy#144] +Condition : ((((isnotnull(d_year#143) AND isnotnull(d_moy#144)) AND (d_year#143 = 2001)) AND (d_moy#144 = 11)) AND isnotnull(d_date_sk#42)) -(135) CometProject -Input [3]: [d_date_sk#42, d_year#131, d_moy#132] +(139) CometProject +Input [3]: [d_date_sk#42, d_year#143, d_moy#144] Arguments: [d_date_sk#42], [d_date_sk#42] -(136) ColumnarToRow [codegen id : 1] +(140) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#42] -(137) BroadcastExchange +(141) BroadcastExchange Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (142) -+- * ColumnarToRow (141) - +- CometProject (140) - +- CometFilter (139) - +- CometScan parquet spark_catalog.default.date_dim (138) +BroadcastExchange (146) ++- * ColumnarToRow (145) + +- CometProject (144) + +- CometFilter (143) + +- CometScan parquet spark_catalog.default.date_dim (142) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#133] +Output [2]: [d_date_sk#25, d_year#145] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(139) CometFilter -Input [2]: [d_date_sk#25, d_year#133] -Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1999)) AND (d_year#133 <= 2001)) AND isnotnull(d_date_sk#25)) +(143) CometFilter +Input [2]: [d_date_sk#25, d_year#145] +Condition : (((isnotnull(d_year#145) AND (d_year#145 >= 1999)) AND (d_year#145 <= 2001)) AND isnotnull(d_date_sk#25)) -(140) CometProject -Input [2]: [d_date_sk#25, d_year#133] +(144) CometProject +Input [2]: [d_date_sk#25, d_year#145] Arguments: [d_date_sk#25], [d_date_sk#25] -(141) ColumnarToRow [codegen id : 1] +(145) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(142) BroadcastExchange +(146) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] @@ -810,12 +835,12 @@ Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN d Subquery:8 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 86 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:9 Hosting operator id = 88 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] -Subquery:10 Hosting operator id = 71 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 103 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:11 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] -Subquery:12 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 90 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index cd494e1546..8d20005f9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -1,214 +1,226 @@ -WholeStageCodegen (76) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] + WholeStageCodegen (80) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] - CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] - CometUnion - CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [sales] - Subquery #3 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count] - CometColumnarExchange #13 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 - RowToColumnar - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow + RowToColumnar + WholeStageCodegen (79) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + InputAdapter + Union + WholeStageCodegen (26) + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] + Filter [sales] + Subquery #3 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + ColumnarToRow + InputAdapter + CometColumnarExchange #13 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #7 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - ColumnarToRow + ReusedExchange [d_date_sk] #7 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #7 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 + RowToColumnar + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + BroadcastExchange #4 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ColumnarToRow + InputAdapter + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #6 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + BroadcastExchange #12 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #6 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - ReusedExchange [d_date_sk] #7 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (52) + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 + RowToColumnar + WholeStageCodegen (51) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (78) + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #15 + RowToColumnar + WholeStageCodegen (77) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #3 - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - RowToColumnar - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #3 - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #15 - RowToColumnar - WholeStageCodegen (75) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 3ce08e1136..2c7c8b099e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == TakeOrderedAndProject (90) +- * BroadcastHashJoin Inner BuildRight (89) - :- * ColumnarToRow (70) - : +- CometFilter (69) - : +- CometHashAggregate (68) + :- * Filter (70) + : +- * HashAggregate (69) + : +- * ColumnarToRow (68) : +- CometColumnarExchange (67) : +- RowToColumnar (66) : +- * HashAggregate (65) @@ -72,9 +72,9 @@ TakeOrderedAndProject (90) : : +- ReusedExchange (57) : +- ReusedExchange (62) +- BroadcastExchange (88) - +- * ColumnarToRow (87) - +- CometFilter (86) - +- CometHashAggregate (85) + +- * Filter (87) + +- * HashAggregate (86) + +- * ColumnarToRow (85) +- CometColumnarExchange (84) +- RowToColumnar (83) +- * HashAggregate (82) @@ -403,113 +403,117 @@ Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) CometHashAggregate +(68) ColumnarToRow [codegen id : 52] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(69) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(69) CometFilter -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(70) ColumnarToRow [codegen id : 52] -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] +(70) Filter [codegen id : 52] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] +Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_sold_date_sk#57 IN dynamicpruning#58)] +PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (72) CometFilter -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Condition : isnotnull(ss_item_sk#54) +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Condition : isnotnull(ss_item_sk#56) (73) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] (74) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] (75) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] +Left keys [1]: [ss_item_sk#56] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None (76) ReusedExchange [Reuses operator id: 59] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] (77) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#59] +Left keys [1]: [ss_item_sk#56] +Right keys [1]: [i_item_sk#61] Join type: Inner Join condition: None (78) Project [codegen id : 50] -Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] +Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] (79) ReusedExchange [Reuses operator id: 130] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#65] (80) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#57] -Right keys [1]: [d_date_sk#63] +Left keys [1]: [ss_sold_date_sk#59] +Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None (81) Project [codegen id : 50] -Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] +Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] +Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] (82) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] +Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] +Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] +Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] (83) RowToColumnar -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] (84) CometColumnarExchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(85) CometHashAggregate -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] +(85) ColumnarToRow [codegen id : 51] +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -(86) CometFilter -Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] -Condition : (isnotnull(sales#71) AND (cast(sales#71 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) +(86) HashAggregate [codegen id : 51] +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72, count(1)#73] +Results [6]: [store AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] -(87) ColumnarToRow [codegen id : 51] -Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +(87) Filter [codegen id : 51] +Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] +Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) (88) BroadcastExchange -Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] (89) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Right keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Join type: Inner Join condition: None (90) TakeOrderedAndProject -Input [12]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51, channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] -Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51, channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] +Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] ===== Subqueries ===== -Subquery:1 Hosting operator id = 69 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* ColumnarToRow (111) -+- CometHashAggregate (110) +Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (111) ++- * ColumnarToRow (110) +- CometColumnarExchange (109) +- RowToColumnar (108) +- * HashAggregate (107) @@ -532,103 +536,105 @@ Subquery:1 Hosting operator id = 69 Hosting Expression = Subquery scalar-subquer (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75] +Output [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_sold_date_sk#75 IN dynamicpruning#76)] +PartitionFilters: [isnotnull(ss_sold_date_sk#79), dynamicpruningexpression(ss_sold_date_sk#79 IN dynamicpruning#80)] ReadSchema: struct (92) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75] +Input [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] (93) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#77] +Output [1]: [d_date_sk#81] (94) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#75] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [ss_sold_date_sk#79] +Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None (95) Project [codegen id : 2] -Output [2]: [ss_quantity#73 AS quantity#78, ss_list_price#74 AS list_price#79] -Input [4]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75, d_date_sk#77] +Output [2]: [ss_quantity#77 AS quantity#82, ss_list_price#78 AS list_price#83] +Input [4]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79, d_date_sk#81] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82] +Output [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#82), dynamicpruningexpression(cs_sold_date_sk#82 IN dynamicpruning#83)] +PartitionFilters: [isnotnull(cs_sold_date_sk#86), dynamicpruningexpression(cs_sold_date_sk#86 IN dynamicpruning#87)] ReadSchema: struct (97) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82] +Input [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] (98) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#84] +Output [1]: [d_date_sk#88] (99) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#82] -Right keys [1]: [d_date_sk#84] +Left keys [1]: [cs_sold_date_sk#86] +Right keys [1]: [d_date_sk#88] Join type: Inner Join condition: None (100) Project [codegen id : 4] -Output [2]: [cs_quantity#80 AS quantity#85, cs_list_price#81 AS list_price#86] -Input [4]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82, d_date_sk#84] +Output [2]: [cs_quantity#84 AS quantity#89, cs_list_price#85 AS list_price#90] +Input [4]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86, d_date_sk#88] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] +Output [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#90)] +PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] ReadSchema: struct (102) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] +Input [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] (103) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#91] +Output [1]: [d_date_sk#95] (104) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#91] +Left keys [1]: [ws_sold_date_sk#93] +Right keys [1]: [d_date_sk#95] Join type: Inner Join condition: None (105) Project [codegen id : 6] -Output [2]: [ws_quantity#87 AS quantity#92, ws_list_price#88 AS list_price#93] -Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#91] +Output [2]: [ws_quantity#91 AS quantity#96, ws_list_price#92 AS list_price#97] +Input [4]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93, d_date_sk#95] (106) Union (107) HashAggregate [codegen id : 7] -Input [2]: [quantity#78, list_price#79] +Input [2]: [quantity#82, list_price#83] Keys: [] -Functions [1]: [partial_avg((cast(quantity#78 as decimal(10,0)) * list_price#79))] -Aggregate Attributes [2]: [sum#94, count#95] -Results [2]: [sum#96, count#97] +Functions [1]: [partial_avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] +Aggregate Attributes [2]: [sum#98, count#99] +Results [2]: [sum#100, count#101] (108) RowToColumnar -Input [2]: [sum#96, count#97] +Input [2]: [sum#100, count#101] (109) CometColumnarExchange -Input [2]: [sum#96, count#97] +Input [2]: [sum#100, count#101] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(110) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#78 as decimal(10,0)) * list_price#79))] +(110) ColumnarToRow [codegen id : 8] +Input [2]: [sum#100, count#101] -(111) ColumnarToRow [codegen id : 8] -Input [1]: [average_sales#98] +(111) HashAggregate [codegen id : 8] +Input [2]: [sum#100, count#101] +Keys: [] +Functions [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] +Aggregate Attributes [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102] +Results [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102 AS average_sales#103] -Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#75 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#79 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#82 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (116) @@ -639,18 +645,18 @@ BroadcastExchange (116) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#42, d_week_seq#99] +Output [2]: [d_date_sk#42, d_week_seq#104] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (113) CometFilter -Input [2]: [d_date_sk#42, d_week_seq#99] -Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#42)) +Input [2]: [d_date_sk#42, d_week_seq#104] +Condition : ((isnotnull(d_week_seq#104) AND (d_week_seq#104 = Subquery scalar-subquery#105, [id=#106])) AND isnotnull(d_date_sk#42)) (114) CometProject -Input [2]: [d_date_sk#42, d_week_seq#99] +Input [2]: [d_date_sk#42, d_week_seq#104] Arguments: [d_date_sk#42], [d_date_sk#42] (115) ColumnarToRow [codegen id : 1] @@ -660,7 +666,7 @@ Input [1]: [d_date_sk#42] Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 113 Hosting Expression = Subquery scalar-subquery#100, [id=#101] +Subquery:6 Hosting operator id = 113 Hosting Expression = Subquery scalar-subquery#105, [id=#106] * ColumnarToRow (120) +- CometProject (119) +- CometFilter (118) @@ -668,22 +674,22 @@ Subquery:6 Hosting operator id = 113 Hosting Expression = Subquery scalar-subque (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] +Output [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (118) CometFilter -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 2000)) AND (d_moy#104 = 12)) AND (d_dom#105 = 11)) +Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] +Condition : (((((isnotnull(d_year#108) AND isnotnull(d_moy#109)) AND isnotnull(d_dom#110)) AND (d_year#108 = 2000)) AND (d_moy#109 = 12)) AND (d_dom#110 = 11)) (119) CometProject -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Arguments: [d_week_seq#102], [d_week_seq#102] +Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] +Arguments: [d_week_seq#107], [d_week_seq#107] (120) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#102] +Input [1]: [d_week_seq#107] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (125) @@ -694,18 +700,18 @@ BroadcastExchange (125) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#106] +Output [2]: [d_date_sk#25, d_year#111] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (122) CometFilter -Input [2]: [d_date_sk#25, d_year#106] -Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1999)) AND (d_year#106 <= 2001)) AND isnotnull(d_date_sk#25)) +Input [2]: [d_date_sk#25, d_year#111] +Condition : (((isnotnull(d_year#111) AND (d_year#111 >= 1999)) AND (d_year#111 <= 2001)) AND isnotnull(d_date_sk#25)) (123) CometProject -Input [2]: [d_date_sk#25, d_year#106] +Input [2]: [d_date_sk#25, d_year#111] Arguments: [d_date_sk#25], [d_date_sk#25] (124) ColumnarToRow [codegen id : 1] @@ -719,9 +725,9 @@ Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN d Subquery:9 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 86 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:10 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 +Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 BroadcastExchange (130) +- * ColumnarToRow (129) +- CometProject (128) @@ -730,28 +736,28 @@ BroadcastExchange (130) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#63, d_week_seq#107] +Output [2]: [d_date_sk#65, d_week_seq#112] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (127) CometFilter -Input [2]: [d_date_sk#63, d_week_seq#107] -Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#63)) +Input [2]: [d_date_sk#65, d_week_seq#112] +Condition : ((isnotnull(d_week_seq#112) AND (d_week_seq#112 = Subquery scalar-subquery#113, [id=#114])) AND isnotnull(d_date_sk#65)) (128) CometProject -Input [2]: [d_date_sk#63, d_week_seq#107] -Arguments: [d_date_sk#63], [d_date_sk#63] +Input [2]: [d_date_sk#65, d_week_seq#112] +Arguments: [d_date_sk#65], [d_date_sk#65] (129) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#63] +Input [1]: [d_date_sk#65] (130) BroadcastExchange -Input [1]: [d_date_sk#63] +Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:12 Hosting operator id = 127 Hosting Expression = Subquery scalar-subquery#108, [id=#109] +Subquery:12 Hosting operator id = 127 Hosting Expression = Subquery scalar-subquery#113, [id=#114] * ColumnarToRow (134) +- CometProject (133) +- CometFilter (132) @@ -759,21 +765,21 @@ Subquery:12 Hosting operator id = 127 Hosting Expression = Subquery scalar-subqu (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] +Output [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (132) CometFilter -Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] -Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1999)) AND (d_moy#112 = 12)) AND (d_dom#113 = 11)) +Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] +Condition : (((((isnotnull(d_year#116) AND isnotnull(d_moy#117)) AND isnotnull(d_dom#118)) AND (d_year#116 = 1999)) AND (d_moy#117 = 12)) AND (d_dom#118 = 11)) (133) CometProject -Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] -Arguments: [d_week_seq#110], [d_week_seq#110] +Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] +Arguments: [d_week_seq#115], [d_week_seq#115] (134) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#110] +Input [1]: [d_week_seq#115] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 13ac319fbd..29d5a8d87c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -1,48 +1,48 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (52) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [sales] - Subquery #4 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count] - CometColumnarExchange #12 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + Filter [sales] + Subquery #4 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + ColumnarToRow + InputAdapter + CometColumnarExchange #12 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 RowToColumnar WholeStageCodegen (25) @@ -169,11 +169,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #13 WholeStageCodegen (51) - ColumnarToRow - InputAdapter - CometFilter [sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + Filter [sales] + ReusedSubquery [average_sales] #4 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 RowToColumnar WholeStageCodegen (50) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt index 135162ca9d..15682a488b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometHashAggregate (22) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- * ColumnarToRow (22) +- CometColumnarExchange (21) +- RowToColumnar (20) +- * HashAggregate (19) @@ -123,17 +123,19 @@ Input [2]: [ca_zip#9, sum#12] Input [2]: [ca_zip#9, sum#12] Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometHashAggregate +(22) ColumnarToRow [codegen id : 5] +Input [2]: [ca_zip#9, sum#12] + +(23) HashAggregate [codegen id : 5] Input [2]: [ca_zip#9, sum#12] Keys [1]: [ca_zip#9] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#13] +Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#13,17,2) AS sum(cs_sales_price)#14] -(23) CometTakeOrderedAndProject -Input [2]: [ca_zip#9, sum(cs_sales_price)#13] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#9 ASC NULLS FIRST], output=[ca_zip#9,sum(cs_sales_price)#13]), 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#13] - -(24) ColumnarToRow [codegen id : 5] -Input [2]: [ca_zip#9, sum(cs_sales_price)#13] +(24) TakeOrderedAndProject +Input [2]: [ca_zip#9, sum(cs_sales_price)#14] +Arguments: 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#14] ===== Subqueries ===== @@ -146,18 +148,18 @@ BroadcastExchange (29) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#14, d_qoy#15] +Output [3]: [d_date_sk#10, d_year#15, d_qoy#16] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (26) CometFilter -Input [3]: [d_date_sk#10, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] +Condition : ((((isnotnull(d_qoy#16) AND isnotnull(d_year#15)) AND (d_qoy#16 = 2)) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#10)) (27) CometProject -Input [3]: [d_date_sk#10, d_year#14, d_qoy#15] +Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] Arguments: [d_date_sk#10], [d_date_sk#10] (28) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index 58d3742879..96fe9a3d78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - CometHashAggregate [ca_zip,sum] +TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] + WholeStageCodegen (5) + HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_zip] #1 RowToColumnar WholeStageCodegen (4) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index cb01af81df..c02e9680b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* ColumnarToRow (47) -+- CometHashAggregate (46) +* HashAggregate (47) ++- * ColumnarToRow (46) +- CometColumnarExchange (45) +- RowToColumnar (44) +- * HashAggregate (43) @@ -256,11 +256,13 @@ Input [3]: [sum#22, sum#23, count#25] Input [3]: [sum#22, sum#23, count#25] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(46) CometHashAggregate +(46) ColumnarToRow [codegen id : 9] +Input [3]: [sum#22, sum#23, count#25] + +(47) HashAggregate [codegen id : 9] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] - -(47) ColumnarToRow [codegen id : 9] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24] +Results [3]: [count(cs_order_number#5)#24 AS order count #26, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#20,17,2) AS total shipping cost #27, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#21,17,2) AS total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index eed6c6d22d..97ea38304d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] + HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (8) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt index 159a82afde..c248a76233 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (45) -+- CometTakeOrderedAndProject (44) - +- CometHashAggregate (43) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- * ColumnarToRow (43) +- CometColumnarExchange (42) +- RowToColumnar (41) +- * HashAggregate (40) @@ -240,17 +240,19 @@ Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_group Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] Arguments: hashpartitioning(i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometHashAggregate +(43) ColumnarToRow [codegen id : 8] +Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] + +(44) HashAggregate [codegen id : 8] Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] Keys [5]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] +Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#61, avg(cast(cs_list_price#5 as decimal(12,2)))#62, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63, avg(cast(cs_sales_price#6 as decimal(12,2)))#64, avg(cast(cs_net_profit#8 as decimal(12,2)))#65, avg(cast(c_birth_year#19 as decimal(12,2)))#66, avg(cast(cd_dep_count#14 as decimal(12,2)))#67] +Results [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, avg(cast(cs_quantity#4 as decimal(12,2)))#61 AS agg1#68, avg(cast(cs_list_price#5 as decimal(12,2)))#62 AS agg2#69, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63 AS agg3#70, avg(cast(cs_sales_price#6 as decimal(12,2)))#64 AS agg4#71, avg(cast(cs_net_profit#8 as decimal(12,2)))#65 AS agg5#72, avg(cast(c_birth_year#19 as decimal(12,2)))#66 AS agg6#73, avg(cast(cd_dep_count#14 as decimal(12,2)))#67 AS agg7#74] -(44) CometTakeOrderedAndProject -Input [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#61, agg2#62, agg3#63, agg4#64, agg5#65, agg6#66, agg7#67] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#29 ASC NULLS FIRST,ca_state#30 ASC NULLS FIRST,ca_county#31 ASC NULLS FIRST,i_item_id#28 ASC NULLS FIRST], output=[i_item_id#28,ca_country#29,ca_state#30,ca_county#31,agg1#61,agg2#62,agg3#63,agg4#64,agg5#65,agg6#66,agg7#67]), 100, [ca_country#29 ASC NULLS FIRST, ca_state#30 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#61, agg2#62, agg3#63, agg4#64, agg5#65, agg6#66, agg7#67] - -(45) ColumnarToRow [codegen id : 8] -Input [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#61, agg2#62, agg3#63, agg4#64, agg5#65, agg6#66, agg7#67] +(45) TakeOrderedAndProject +Input [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] +Arguments: 100, [ca_country#29 ASC NULLS FIRST, ca_state#30 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] ===== Subqueries ===== @@ -263,18 +265,18 @@ BroadcastExchange (50) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#68] +Output [2]: [d_date_sk#25, d_year#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct (47) CometFilter -Input [2]: [d_date_sk#25, d_year#68] -Condition : ((isnotnull(d_year#68) AND (d_year#68 = 1998)) AND isnotnull(d_date_sk#25)) +Input [2]: [d_date_sk#25, d_year#75] +Condition : ((isnotnull(d_year#75) AND (d_year#75 = 1998)) AND isnotnull(d_date_sk#25)) (48) CometProject -Input [2]: [d_date_sk#25, d_year#68] +Input [2]: [d_date_sk#25, d_year#75] Arguments: [d_date_sk#25], [d_date_sk#25] (49) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt index 0f94dd57c4..06e0f4d108 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] +TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + WholeStageCodegen (8) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 RowToColumnar WholeStageCodegen (7) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt index 5f4b64a18d..894b5cb6b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (41) -+- CometTakeOrderedAndProject (40) - +- CometHashAggregate (39) +TakeOrderedAndProject (41) ++- * HashAggregate (40) + +- * ColumnarToRow (39) +- CometColumnarExchange (38) +- RowToColumnar (37) +- * HashAggregate (36) @@ -219,15 +219,17 @@ Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(39) CometHashAggregate +(39) ColumnarToRow [codegen id : 7] +Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] + +(40) HashAggregate [codegen id : 7] Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] Keys [4]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#23] +Results [5]: [i_brand_id#10 AS brand_id#24, i_brand#11 AS brand#25, i_manufact_id#12, i_manufact#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#23,17,2) AS ext_price#26] -(40) CometTakeOrderedAndProject -Input [5]: [brand_id#23, brand#24, i_manufact_id#12, i_manufact#13, ext_price#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#25 DESC NULLS LAST,brand#24 ASC NULLS FIRST,brand_id#23 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#13 ASC NULLS FIRST], output=[brand_id#23,brand#24,i_manufact_id#12,i_manufact#13,ext_price#25]), 100, [ext_price#25 DESC NULLS LAST, brand#24 ASC NULLS FIRST, brand_id#23 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#23, brand#24, i_manufact_id#12, i_manufact#13, ext_price#25] - -(41) ColumnarToRow [codegen id : 7] -Input [5]: [brand_id#23, brand#24, i_manufact_id#12, i_manufact#13, ext_price#25] +(41) TakeOrderedAndProject +Input [5]: [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] +Arguments: 100, [ext_price#26 DESC NULLS LAST, brand#25 ASC NULLS FIRST, brand_id#24 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt index 5b9f4956e5..3a6991bbb5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] +TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] + WholeStageCodegen (7) + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 RowToColumnar WholeStageCodegen (6) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt index 7974fb4eeb..bbd6ff1aed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt @@ -7,8 +7,8 @@ +- * BroadcastHashJoin Inner BuildRight (36) :- * Project (24) : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * ColumnarToRow (17) - : : +- CometHashAggregate (16) + : :- * HashAggregate (17) + : : +- * ColumnarToRow (16) : : +- CometColumnarExchange (15) : : +- RowToColumnar (14) : : +- * HashAggregate (13) @@ -32,8 +32,8 @@ +- BroadcastExchange (35) +- * Project (34) +- * BroadcastHashJoin Inner BuildRight (33) - :- * ColumnarToRow (27) - : +- CometHashAggregate (26) + :- * HashAggregate (27) + : +- * ColumnarToRow (26) : +- ReusedExchange (25) +- BroadcastExchange (32) +- * ColumnarToRow (31) @@ -113,114 +113,118 @@ Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#2 Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 8] +Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] + +(17) HashAggregate [codegen id : 8] Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] - -(17) ColumnarToRow [codegen id : 8] -Input [8]: [d_week_seq#10, sun_sales#26, mon_sales#27, tue_sales#28, wed_sales#29, thu_sales#30, fri_sales#31, sat_sales#32] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26,17,2) AS sun_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27,17,2) AS mon_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28,17,2) AS tue_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29,17,2) AS wed_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30,17,2) AS thu_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31,17,2) AS fri_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32,17,2) AS sat_sales#39] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#33, d_year#34] +Output [2]: [d_week_seq#40, d_year#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] ReadSchema: struct (19) CometFilter -Input [2]: [d_week_seq#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2001)) AND isnotnull(d_week_seq#33)) +Input [2]: [d_week_seq#40, d_year#41] +Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2001)) AND isnotnull(d_week_seq#40)) (20) CometProject -Input [2]: [d_week_seq#33, d_year#34] -Arguments: [d_week_seq#33], [d_week_seq#33] +Input [2]: [d_week_seq#40, d_year#41] +Arguments: [d_week_seq#40], [d_week_seq#40] (21) ColumnarToRow [codegen id : 3] -Input [1]: [d_week_seq#33] +Input [1]: [d_week_seq#40] (22) BroadcastExchange -Input [1]: [d_week_seq#33] +Input [1]: [d_week_seq#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] (23) BroadcastHashJoin [codegen id : 8] Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#33] +Right keys [1]: [d_week_seq#40] Join type: Inner Join condition: None (24) Project [codegen id : 8] -Output [8]: [d_week_seq#10 AS d_week_seq1#35, sun_sales#26 AS sun_sales1#36, mon_sales#27 AS mon_sales1#37, tue_sales#28 AS tue_sales1#38, wed_sales#29 AS wed_sales1#39, thu_sales#30 AS thu_sales1#40, fri_sales#31 AS fri_sales1#41, sat_sales#32 AS sat_sales1#42] -Input [9]: [d_week_seq#10, sun_sales#26, mon_sales#27, tue_sales#28, wed_sales#29, thu_sales#30, fri_sales#31, sat_sales#32, d_week_seq#33] +Output [8]: [d_week_seq#10 AS d_week_seq1#42, sun_sales#33 AS sun_sales1#43, mon_sales#34 AS mon_sales1#44, tue_sales#35 AS tue_sales1#45, wed_sales#36 AS wed_sales1#46, thu_sales#37 AS thu_sales1#47, fri_sales#38 AS fri_sales1#48, sat_sales#39 AS sat_sales1#49] +Input [9]: [d_week_seq#10, sun_sales#33, mon_sales#34, tue_sales#35, wed_sales#36, thu_sales#37, fri_sales#38, sat_sales#39, d_week_seq#40] (25) ReusedExchange [Reuses operator id: 15] -Output [8]: [d_week_seq#10, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] +Output [8]: [d_week_seq#10, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] -(26) CometHashAggregate -Input [8]: [d_week_seq#10, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] +(26) ColumnarToRow [codegen id : 7] +Input [8]: [d_week_seq#10, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] + +(27) HashAggregate [codegen id : 7] +Input [8]: [d_week_seq#10, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] - -(27) ColumnarToRow [codegen id : 7] -Input [8]: [d_week_seq#10, sun_sales#26, mon_sales#27, tue_sales#28, wed_sales#29, thu_sales#30, fri_sales#31, sat_sales#32] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26,17,2) AS sun_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27,17,2) AS mon_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28,17,2) AS tue_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29,17,2) AS wed_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30,17,2) AS thu_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31,17,2) AS fri_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32,17,2) AS sat_sales#39] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#50, d_year#51] +Output [2]: [d_week_seq#57, d_year#58] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct (29) CometFilter -Input [2]: [d_week_seq#50, d_year#51] -Condition : ((isnotnull(d_year#51) AND (d_year#51 = 2002)) AND isnotnull(d_week_seq#50)) +Input [2]: [d_week_seq#57, d_year#58] +Condition : ((isnotnull(d_year#58) AND (d_year#58 = 2002)) AND isnotnull(d_week_seq#57)) (30) CometProject -Input [2]: [d_week_seq#50, d_year#51] -Arguments: [d_week_seq#50], [d_week_seq#50] +Input [2]: [d_week_seq#57, d_year#58] +Arguments: [d_week_seq#57], [d_week_seq#57] (31) ColumnarToRow [codegen id : 6] -Input [1]: [d_week_seq#50] +Input [1]: [d_week_seq#57] (32) BroadcastExchange -Input [1]: [d_week_seq#50] +Input [1]: [d_week_seq#57] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] (33) BroadcastHashJoin [codegen id : 7] Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#50] +Right keys [1]: [d_week_seq#57] Join type: Inner Join condition: None (34) Project [codegen id : 7] -Output [8]: [d_week_seq#10 AS d_week_seq2#52, sun_sales#26 AS sun_sales2#53, mon_sales#27 AS mon_sales2#54, tue_sales#28 AS tue_sales2#55, wed_sales#29 AS wed_sales2#56, thu_sales#30 AS thu_sales2#57, fri_sales#31 AS fri_sales2#58, sat_sales#32 AS sat_sales2#59] -Input [9]: [d_week_seq#10, sun_sales#26, mon_sales#27, tue_sales#28, wed_sales#29, thu_sales#30, fri_sales#31, sat_sales#32, d_week_seq#50] +Output [8]: [d_week_seq#10 AS d_week_seq2#59, sun_sales#33 AS sun_sales2#60, mon_sales#34 AS mon_sales2#61, tue_sales#35 AS tue_sales2#62, wed_sales#36 AS wed_sales2#63, thu_sales#37 AS thu_sales2#64, fri_sales#38 AS fri_sales2#65, sat_sales#39 AS sat_sales2#66] +Input [9]: [d_week_seq#10, sun_sales#33, mon_sales#34, tue_sales#35, wed_sales#36, thu_sales#37, fri_sales#38, sat_sales#39, d_week_seq#57] (35) BroadcastExchange -Input [8]: [d_week_seq2#52, sun_sales2#53, mon_sales2#54, tue_sales2#55, wed_sales2#56, thu_sales2#57, fri_sales2#58, sat_sales2#59] +Input [8]: [d_week_seq2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=5] (36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [d_week_seq1#35] -Right keys [1]: [(d_week_seq2#52 - 53)] +Left keys [1]: [d_week_seq1#42] +Right keys [1]: [(d_week_seq2#59 - 53)] Join type: Inner Join condition: None (37) Project [codegen id : 8] -Output [8]: [d_week_seq1#35, round((sun_sales1#36 / sun_sales2#53), 2) AS round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1#37 / mon_sales2#54), 2) AS round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1#38 / tue_sales2#55), 2) AS round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1#39 / wed_sales2#56), 2) AS round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1#40 / thu_sales2#57), 2) AS round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1#41 / fri_sales2#58), 2) AS round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1#42 / sat_sales2#59), 2) AS round((sat_sales1 / sat_sales2), 2)#66] -Input [16]: [d_week_seq1#35, sun_sales1#36, mon_sales1#37, tue_sales1#38, wed_sales1#39, thu_sales1#40, fri_sales1#41, sat_sales1#42, d_week_seq2#52, sun_sales2#53, mon_sales2#54, tue_sales2#55, wed_sales2#56, thu_sales2#57, fri_sales2#58, sat_sales2#59] +Output [8]: [d_week_seq1#42, round((sun_sales1#43 / sun_sales2#60), 2) AS round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1#44 / mon_sales2#61), 2) AS round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1#45 / tue_sales2#62), 2) AS round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1#46 / wed_sales2#63), 2) AS round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1#47 / thu_sales2#64), 2) AS round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1#48 / fri_sales2#65), 2) AS round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1#49 / sat_sales2#66), 2) AS round((sat_sales1 / sat_sales2), 2)#73] +Input [16]: [d_week_seq1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] (38) RowToColumnar -Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66] +Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] (39) CometColumnarExchange -Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66] -Arguments: rangepartitioning(d_week_seq1#35 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] +Arguments: rangepartitioning(d_week_seq1#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] (40) CometSort -Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66] -Arguments: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66], [d_week_seq1#35 ASC NULLS FIRST] +Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] +Arguments: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73], [d_week_seq1#42 ASC NULLS FIRST] (41) ColumnarToRow [codegen id : 9] -Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66] +Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index f4b68cfee2..d4e1fa96a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -9,9 +9,9 @@ WholeStageCodegen (9) BroadcastHashJoin [d_week_seq1,d_week_seq2] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [d_week_seq] #2 RowToColumnar WholeStageCodegen (2) @@ -45,9 +45,9 @@ WholeStageCodegen (9) WholeStageCodegen (7) Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 InputAdapter BroadcastExchange #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index 5d279243c5..07df1fbf05 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -1,26 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * Project (21) - +- Window (20) - +- * ColumnarToRow (19) - +- CometSort (18) - +- CometColumnarExchange (17) - +- CometHashAggregate (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * ColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- RowToColumnar (18) + +- * HashAggregate (17) + +- * ColumnarToRow (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -66,7 +68,7 @@ Join condition: None Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 27] +(10) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -93,63 +95,71 @@ Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] +Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] + +(18) RowToColumnar +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -(17) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +(19) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) CometSort -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] -Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] +(20) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 4] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +(21) ColumnarToRow [codegen id : 5] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -(20) Window -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] -Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] +(22) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(21) Project [codegen id : 5] -Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] -Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] +(23) Project [codegen id : 6] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] -(22) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +(24) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#18] +Output [2]: [d_date_sk#11, d_date#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [d_date_sk#11, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(26) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(25) CometProject -Input [2]: [d_date_sk#11, d_date#18] +(27) CometProject +Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index 202a09b6d3..2b21c3ce4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -1,40 +1,44 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (5) + WholeStageCodegen (6) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (4) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometSort [i_class] CometColumnarExchange [i_class] #1 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index 5c69eb848b..f559933927 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* ColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometFilter (24) - +- CometHashAggregate (23) +TakeOrderedAndProject (26) ++- * Filter (25) + +- * HashAggregate (24) + +- * ColumnarToRow (23) +- CometColumnarExchange (22) +- RowToColumnar (21) +- * HashAggregate (20) @@ -129,21 +129,23 @@ Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometHashAggregate +(23) ColumnarToRow [codegen id : 5] +Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] + +(24) HashAggregate [codegen id : 5] Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] Keys [2]: [w_warehouse_name#7, i_item_id#9] Functions [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] +Aggregate Attributes [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18] +Results [4]: [w_warehouse_name#7, i_item_id#9, sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17 AS inv_before#19, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_after#20] -(24) CometFilter -Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18] -Condition : (CASE WHEN (inv_before#17 > 0) THEN ((cast(inv_after#18 as double) / cast(inv_before#17 as double)) >= 0.666667) END AND CASE WHEN (inv_before#17 > 0) THEN ((cast(inv_after#18 as double) / cast(inv_before#17 as double)) <= 1.5) END) - -(25) CometTakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#9 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#9,inv_before#17,inv_after#18]), 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18] +(25) Filter [codegen id : 5] +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] +Condition : (CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667) END AND CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5) END) -(26) ColumnarToRow [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18] +(26) TakeOrderedAndProject +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] +Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index 3556650ca1..016ab7b91b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - CometFilter [inv_before,inv_after] - CometHashAggregate [w_warehouse_name,i_item_id,sum,sum] +TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] + WholeStageCodegen (5) + Filter [inv_before,inv_after] + HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [w_warehouse_name,i_item_id] #1 RowToColumnar WholeStageCodegen (4) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt index 13689dc21e..0513ff1811 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- CometHashAggregate (23) +TakeOrderedAndProject (25) ++- * HashAggregate (24) + +- * ColumnarToRow (23) +- CometColumnarExchange (22) +- RowToColumnar (21) +- * HashAggregate (20) @@ -128,17 +128,19 @@ Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grou Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] Arguments: hashpartitioning(i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometHashAggregate +(23) ColumnarToRow [codegen id : 5] +Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] + +(24) HashAggregate [codegen id : 5] Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] Keys [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#22] +Results [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, avg(inv_quantity_on_hand#3)#22 AS qoh#23] -(24) CometTakeOrderedAndProject -Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#22] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#22 ASC NULLS FIRST,i_product_name#13 ASC NULLS FIRST,i_brand#14 ASC NULLS FIRST,i_class#15 ASC NULLS FIRST,i_category#16 ASC NULLS FIRST], output=[i_product_name#13,i_brand#14,i_class#15,i_category#16,qoh#22]), 100, [qoh#22 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#22] - -(25) ColumnarToRow [codegen id : 5] -Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#22] +(25) TakeOrderedAndProject +Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] +Arguments: 100, [qoh#23 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] ===== Subqueries ===== @@ -151,18 +153,18 @@ BroadcastExchange (30) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#23] +Output [2]: [d_date_sk#6, d_month_seq#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct (27) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#23] -Condition : (((isnotnull(d_month_seq#23) AND (d_month_seq#23 >= 1200)) AND (d_month_seq#23 <= 1211)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_month_seq#24] +Condition : (((isnotnull(d_month_seq#24) AND (d_month_seq#24 >= 1200)) AND (d_month_seq#24 <= 1211)) AND isnotnull(d_date_sk#6)) (28) CometProject -Input [2]: [d_date_sk#6, d_month_seq#23] +Input [2]: [d_date_sk#6, d_month_seq#24] Arguments: [d_date_sk#6], [d_date_sk#6] (29) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt index b0c2728e3a..cfa2bd703b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] + WholeStageCodegen (5) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 RowToColumnar WholeStageCodegen (4) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index 973532dc28..763f886869 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -1,79 +1,81 @@ == Physical Plan == -* ColumnarToRow (75) -+- CometHashAggregate (74) - +- CometColumnarExchange (73) - +- RowToColumnar (72) - +- * HashAggregate (71) - +- Union (70) - :- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * ColumnarToRow (47) - : : +- CometProject (46) - : : +- CometSortMergeJoin (45) - : : :- CometSort (27) - : : : +- CometColumnarExchange (26) - : : : +- RowToColumnar (25) - : : : +- * Project (24) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : : :- * ColumnarToRow (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (22) - : : : +- * ColumnarToRow (21) - : : : +- CometProject (20) - : : : +- CometFilter (19) - : : : +- CometHashAggregate (18) - : : : +- CometColumnarExchange (17) - : : : +- RowToColumnar (16) - : : : +- * HashAggregate (15) - : : : +- * Project (14) - : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : :- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (12) - : : : +- * ColumnarToRow (11) - : : : +- CometFilter (10) - : : : +- CometScan parquet spark_catalog.default.item (9) - : : +- CometSort (44) - : : +- CometProject (43) - : : +- CometFilter (42) - : : +- CometHashAggregate (41) - : : +- CometColumnarExchange (40) - : : +- RowToColumnar (39) - : : +- * HashAggregate (38) - : : +- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * ColumnarToRow (31) - : : : +- CometProject (30) - : : : +- CometFilter (29) - : : : +- CometScan parquet spark_catalog.default.store_sales (28) - : : +- BroadcastExchange (35) - : : +- * ColumnarToRow (34) - : : +- CometFilter (33) - : : +- CometScan parquet spark_catalog.default.customer (32) - : +- ReusedExchange (48) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * ColumnarToRow (66) - : +- CometProject (65) - : +- CometSortMergeJoin (64) - : :- CometSort (58) - : : +- CometColumnarExchange (57) - : : +- RowToColumnar (56) - : : +- * Project (55) - : : +- * BroadcastHashJoin LeftSemi BuildRight (54) - : : :- * ColumnarToRow (52) - : : : +- CometScan parquet spark_catalog.default.web_sales (51) - : : +- ReusedExchange (53) - : +- CometSort (63) - : +- CometProject (62) - : +- CometFilter (61) - : +- CometHashAggregate (60) - : +- ReusedExchange (59) - +- ReusedExchange (67) +* HashAggregate (77) ++- * ColumnarToRow (76) + +- CometColumnarExchange (75) + +- RowToColumnar (74) + +- * HashAggregate (73) + +- Union (72) + :- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Project (48) + : : +- * SortMergeJoin LeftSemi (47) + : : :- * ColumnarToRow (28) + : : : +- CometSort (27) + : : : +- CometColumnarExchange (26) + : : : +- RowToColumnar (25) + : : : +- * Project (24) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : : :- * ColumnarToRow (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (22) + : : : +- * Project (21) + : : : +- * Filter (20) + : : : +- * HashAggregate (19) + : : : +- * ColumnarToRow (18) + : : : +- CometColumnarExchange (17) + : : : +- RowToColumnar (16) + : : : +- * HashAggregate (15) + : : : +- * Project (14) + : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : :- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (12) + : : : +- * ColumnarToRow (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.item (9) + : : +- * Sort (46) + : : +- * Project (45) + : : +- * Filter (44) + : : +- * HashAggregate (43) + : : +- * ColumnarToRow (42) + : : +- CometColumnarExchange (41) + : : +- RowToColumnar (40) + : : +- * HashAggregate (39) + : : +- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * ColumnarToRow (32) + : : : +- CometProject (31) + : : : +- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.store_sales (29) + : : +- BroadcastExchange (36) + : : +- * ColumnarToRow (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.customer (33) + : +- ReusedExchange (49) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (68) + : +- * SortMergeJoin LeftSemi (67) + : :- * ColumnarToRow (60) + : : +- CometSort (59) + : : +- CometColumnarExchange (58) + : : +- RowToColumnar (57) + : : +- * Project (56) + : : +- * BroadcastHashJoin LeftSemi BuildRight (55) + : : :- * ColumnarToRow (53) + : : : +- CometScan parquet spark_catalog.default.web_sales (52) + : : +- ReusedExchange (54) + : +- * Sort (66) + : +- * Project (65) + : +- * Filter (64) + : +- * HashAggregate (63) + : +- * ColumnarToRow (62) + : +- ReusedExchange (61) + +- ReusedExchange (69) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -101,7 +103,7 @@ Condition : isnotnull(ss_item_sk#7) (5) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(6) ReusedExchange [Reuses operator id: 85] +(6) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#10, d_date#11] (7) BroadcastHashJoin [codegen id : 3] @@ -156,29 +158,31 @@ Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(18) CometHashAggregate +(18) ColumnarToRow [codegen id : 4] +Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] + +(19) HashAggregate [codegen id : 4] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#17] +Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(19) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) +(20) Filter [codegen id : 4] +Input [2]: [item_sk#18, cnt#19] +Condition : (cnt#19 > 4) -(20) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(21) ColumnarToRow [codegen id : 4] -Input [1]: [item_sk#17] +(21) Project [codegen id : 4] +Output [1]: [item_sk#18] +Input [2]: [item_sk#18, cnt#19] (22) BroadcastExchange -Input [1]: [item_sk#17] +Input [1]: [item_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] (23) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#17] +Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None @@ -197,402 +201,430 @@ Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, Come Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] +(28) ColumnarToRow [codegen id : 6] +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(29) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) +(30) CometFilter +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#20) -(30) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] +(31) CometProject +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -(31) ColumnarToRow [codegen id : 7] -Input [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] +(32) ColumnarToRow [codegen id : 8] +Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] (unknown) Scan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] +Output [1]: [c_customer_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(33) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) +(34) CometFilter +Input [1]: [c_customer_sk#24] +Condition : isnotnull(c_customer_sk#24) -(34) ColumnarToRow [codegen id : 6] -Input [1]: [c_customer_sk#23] +(35) ColumnarToRow [codegen id : 7] +Input [1]: [c_customer_sk#24] -(35) BroadcastExchange -Input [1]: [c_customer_sk#23] +(36) BroadcastExchange +Input [1]: [c_customer_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_customer_sk#19] -Right keys [1]: [c_customer_sk#23] +(37) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_customer_sk#20] +Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(37) Project [codegen id : 7] -Output [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(38) HashAggregate [codegen id : 7] -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] -Aggregate Attributes [2]: [sum#24, isEmpty#25] -Results [3]: [c_customer_sk#23, sum#26, isEmpty#27] - -(39) RowToColumnar -Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] - -(40) CometColumnarExchange -Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(41) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(42) CometFilter -Input [2]: [c_customer_sk#23, ssales#28] -Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#29, [id=#30]))) - -(43) CometProject -Input [2]: [c_customer_sk#23, ssales#28] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(44) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(45) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi +(38) Project [codegen id : 8] +Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] + +(39) HashAggregate [codegen id : 8] +Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Keys [1]: [c_customer_sk#24] +Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [2]: [sum#25, isEmpty#26] +Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] + +(40) RowToColumnar +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] + +(41) CometColumnarExchange +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(42) ColumnarToRow [codegen id : 9] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] + +(43) HashAggregate [codegen id : 9] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Keys [1]: [c_customer_sk#24] +Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] + +(44) Filter [codegen id : 9] +Input [2]: [c_customer_sk#24, ssales#30] +Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) + +(45) Project [codegen id : 9] +Output [1]: [c_customer_sk#24] +Input [2]: [c_customer_sk#24, ssales#30] + +(46) Sort [codegen id : 9] +Input [1]: [c_customer_sk#24] +Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 + +(47) SortMergeJoin [codegen id : 11] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#24] +Join type: LeftSemi +Join condition: None -(46) CometProject +(48) Project [codegen id : 11] +Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(47) ColumnarToRow [codegen id : 9] -Input [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(48) ReusedExchange [Reuses operator id: 80] -Output [1]: [d_date_sk#31] +(49) ReusedExchange [Reuses operator id: 82] +Output [1]: [d_date_sk#33] -(49) BroadcastHashJoin [codegen id : 9] +(50) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#31] +Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(50) Project [codegen id : 9] -Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#31] +(51) Project [codegen id : 11] +Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#34] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#33] (unknown) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Output [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ws_sold_date_sk#39), dynamicpruningexpression(ws_sold_date_sk#39 IN dynamicpruning#40)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 14] -Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +(53) ColumnarToRow [codegen id : 16] +Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(53) ReusedExchange [Reuses operator id: 22] -Output [1]: [item_sk#17] +(54) ReusedExchange [Reuses operator id: 22] +Output [1]: [item_sk#18] -(54) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [item_sk#17] +(55) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(55) Project [codegen id : 14] -Output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +(56) Project [codegen id : 16] +Output [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(56) RowToColumnar -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +(57) RowToColumnar +Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(57) CometColumnarExchange -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(58) CometColumnarExchange +Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Arguments: hashpartitioning(ws_bill_customer_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(58) CometSort -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34 ASC NULLS FIRST] +(59) CometSort +Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Arguments: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39], [ws_bill_customer_sk#36 ASC NULLS FIRST] -(59) ReusedExchange [Reuses operator id: 40] -Output [3]: [c_customer_sk#23, sum#26, isEmpty#27] +(60) ColumnarToRow [codegen id : 17] +Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(60) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] +(61) ReusedExchange [Reuses operator id: 41] +Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(61) CometFilter -Input [2]: [c_customer_sk#23, ssales#28] -Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30]))) +(62) ColumnarToRow [codegen id : 20] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(62) CometProject -Input [2]: [c_customer_sk#23, ssales#28] -Arguments: [c_customer_sk#23], [c_customer_sk#23] +(63) HashAggregate [codegen id : 20] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Keys [1]: [c_customer_sk#24] +Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(63) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] +(64) Filter [codegen id : 20] +Input [2]: [c_customer_sk#24, ssales#30] +Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(64) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [c_customer_sk#23] -Arguments: [ws_bill_customer_sk#34], [c_customer_sk#23], LeftSemi +(65) Project [codegen id : 20] +Output [1]: [c_customer_sk#24] +Input [2]: [c_customer_sk#24, ssales#30] -(65) CometProject -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +(66) Sort [codegen id : 20] +Input [1]: [c_customer_sk#24] +Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 + +(67) SortMergeJoin [codegen id : 22] +Left keys [1]: [ws_bill_customer_sk#36] +Right keys [1]: [c_customer_sk#24] +Join type: LeftSemi +Join condition: None -(66) ColumnarToRow [codegen id : 18] -Input [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +(68) Project [codegen id : 22] +Output [3]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(67) ReusedExchange [Reuses operator id: 80] -Output [1]: [d_date_sk#39] +(69) ReusedExchange [Reuses operator id: 82] +Output [1]: [d_date_sk#41] -(68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#39] +(70) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_sold_date_sk#39] +Right keys [1]: [d_date_sk#41] Join type: Inner Join condition: None -(69) Project [codegen id : 18] -Output [1]: [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#40] -Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#39] +(71) Project [codegen id : 22] +Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#42] +Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#41] -(70) Union +(72) Union -(71) HashAggregate [codegen id : 19] -Input [1]: [sales#32] +(73) HashAggregate [codegen id : 23] +Input [1]: [sales#34] Keys: [] -Functions [1]: [partial_sum(sales#32)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [2]: [sum#43, isEmpty#44] +Functions [1]: [partial_sum(sales#34)] +Aggregate Attributes [2]: [sum#43, isEmpty#44] +Results [2]: [sum#45, isEmpty#46] -(72) RowToColumnar -Input [2]: [sum#43, isEmpty#44] +(74) RowToColumnar +Input [2]: [sum#45, isEmpty#46] -(73) CometColumnarExchange -Input [2]: [sum#43, isEmpty#44] +(75) CometColumnarExchange +Input [2]: [sum#45, isEmpty#46] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(74) CometHashAggregate -Input [2]: [sum#43, isEmpty#44] -Keys: [] -Functions [1]: [sum(sales#32)] +(76) ColumnarToRow [codegen id : 24] +Input [2]: [sum#45, isEmpty#46] -(75) ColumnarToRow [codegen id : 20] -Input [1]: [sum(sales)#45] +(77) HashAggregate [codegen id : 24] +Input [2]: [sum#45, isEmpty#46] +Keys: [] +Functions [1]: [sum(sales#34)] +Aggregate Attributes [1]: [sum(sales#34)#47] +Results [1]: [sum(sales#34)#47 AS sum(sales)#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (80) -+- * ColumnarToRow (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +BroadcastExchange (82) ++- * ColumnarToRow (81) + +- CometProject (80) + +- CometFilter (79) + +- CometScan parquet spark_catalog.default.date_dim (78) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#31, d_year#46, d_moy#47] +Output [3]: [d_date_sk#33, d_year#49, d_moy#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter -Input [3]: [d_date_sk#31, d_year#46, d_moy#47] -Condition : ((((isnotnull(d_year#46) AND isnotnull(d_moy#47)) AND (d_year#46 = 2000)) AND (d_moy#47 = 2)) AND isnotnull(d_date_sk#31)) +(79) CometFilter +Input [3]: [d_date_sk#33, d_year#49, d_moy#50] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2000)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#33)) -(78) CometProject -Input [3]: [d_date_sk#31, d_year#46, d_moy#47] -Arguments: [d_date_sk#31], [d_date_sk#31] +(80) CometProject +Input [3]: [d_date_sk#33, d_year#49, d_moy#50] +Arguments: [d_date_sk#33], [d_date_sk#33] -(79) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#31] +(81) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#33] -(80) BroadcastExchange -Input [1]: [d_date_sk#31] +(82) BroadcastExchange +Input [1]: [d_date_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (85) -+- * ColumnarToRow (84) - +- CometProject (83) - +- CometFilter (82) - +- CometScan parquet spark_catalog.default.date_dim (81) +BroadcastExchange (87) ++- * ColumnarToRow (86) + +- CometProject (85) + +- CometFilter (84) + +- CometScan parquet spark_catalog.default.date_dim (83) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#48] +Output [3]: [d_date_sk#10, d_date#11, d_year#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(82) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#48] -Condition : (d_year#48 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +(84) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#51] +Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(83) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#48] +(85) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#51] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(84) ColumnarToRow [codegen id : 1] +(86) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(85) BroadcastExchange +(87) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#29, [id=#30] -* ColumnarToRow (102) -+- CometHashAggregate (101) - +- CometColumnarExchange (100) - +- CometHashAggregate (99) - +- CometHashAggregate (98) - +- CometColumnarExchange (97) - +- RowToColumnar (96) - +- * HashAggregate (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * ColumnarToRow (88) - : : +- CometFilter (87) - : : +- CometScan parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (89) - +- ReusedExchange (92) +Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +* HashAggregate (106) ++- * ColumnarToRow (105) + +- CometColumnarExchange (104) + +- RowToColumnar (103) + +- * HashAggregate (102) + +- * HashAggregate (101) + +- * ColumnarToRow (100) + +- CometColumnarExchange (99) + +- RowToColumnar (98) + +- * HashAggregate (97) + +- * Project (96) + +- * BroadcastHashJoin Inner BuildRight (95) + :- * Project (93) + : +- * BroadcastHashJoin Inner BuildRight (92) + : :- * ColumnarToRow (90) + : : +- CometFilter (89) + : : +- CometScan parquet spark_catalog.default.store_sales (88) + : +- ReusedExchange (91) + +- ReusedExchange (94) (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52] +Output [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#52), dynamicpruningexpression(ss_sold_date_sk#52 IN dynamicpruning#53)] +PartitionFilters: [isnotnull(ss_sold_date_sk#55), dynamicpruningexpression(ss_sold_date_sk#55 IN dynamicpruning#56)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(87) CometFilter -Input [4]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52] -Condition : isnotnull(ss_customer_sk#49) +(89) CometFilter +Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] +Condition : isnotnull(ss_customer_sk#52) -(88) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52] +(90) ColumnarToRow [codegen id : 3] +Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] -(89) ReusedExchange [Reuses operator id: 35] -Output [1]: [c_customer_sk#54] +(91) ReusedExchange [Reuses operator id: 36] +Output [1]: [c_customer_sk#57] -(90) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#49] -Right keys [1]: [c_customer_sk#54] +(92) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_customer_sk#52] +Right keys [1]: [c_customer_sk#57] Join type: Inner Join condition: None -(91) Project [codegen id : 3] -Output [4]: [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54] -Input [5]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54] +(93) Project [codegen id : 3] +Output [4]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] +Input [5]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] -(92) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#55] +(94) ReusedExchange [Reuses operator id: 111] +Output [1]: [d_date_sk#58] -(93) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#52] -Right keys [1]: [d_date_sk#55] +(95) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#55] +Right keys [1]: [d_date_sk#58] Join type: Inner Join condition: None -(94) Project [codegen id : 3] -Output [3]: [ss_quantity#50, ss_sales_price#51, c_customer_sk#54] -Input [5]: [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54, d_date_sk#55] +(96) Project [codegen id : 3] +Output [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] +Input [5]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57, d_date_sk#58] + +(97) HashAggregate [codegen id : 3] +Input [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] +Keys [1]: [c_customer_sk#57] +Functions [1]: [partial_sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] +Aggregate Attributes [2]: [sum#59, isEmpty#60] +Results [3]: [c_customer_sk#57, sum#61, isEmpty#62] -(95) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#50, ss_sales_price#51, c_customer_sk#54] -Keys [1]: [c_customer_sk#54] -Functions [1]: [partial_sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))] -Aggregate Attributes [2]: [sum#56, isEmpty#57] -Results [3]: [c_customer_sk#54, sum#58, isEmpty#59] +(98) RowToColumnar +Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -(96) RowToColumnar -Input [3]: [c_customer_sk#54, sum#58, isEmpty#59] +(99) CometColumnarExchange +Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] +Arguments: hashpartitioning(c_customer_sk#57, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(97) CometColumnarExchange -Input [3]: [c_customer_sk#54, sum#58, isEmpty#59] -Arguments: hashpartitioning(c_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(100) ColumnarToRow [codegen id : 4] +Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -(98) CometHashAggregate -Input [3]: [c_customer_sk#54, sum#58, isEmpty#59] -Keys [1]: [c_customer_sk#54] -Functions [1]: [sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))] +(101) HashAggregate [codegen id : 4] +Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] +Keys [1]: [c_customer_sk#57] +Functions [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63] +Results [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63 AS csales#64] -(99) CometHashAggregate -Input [1]: [csales#60] +(102) HashAggregate [codegen id : 4] +Input [1]: [csales#64] Keys: [] -Functions [1]: [partial_max(csales#60)] +Functions [1]: [partial_max(csales#64)] +Aggregate Attributes [1]: [max#65] +Results [1]: [max#66] -(100) CometColumnarExchange -Input [1]: [max#61] +(103) RowToColumnar +Input [1]: [max#66] + +(104) CometColumnarExchange +Input [1]: [max#66] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(101) CometHashAggregate -Input [1]: [max#61] -Keys: [] -Functions [1]: [max(csales#60)] +(105) ColumnarToRow [codegen id : 5] +Input [1]: [max#66] -(102) ColumnarToRow [codegen id : 4] -Input [1]: [tpcds_cmax#62] +(106) HashAggregate [codegen id : 5] +Input [1]: [max#66] +Keys: [] +Functions [1]: [max(csales#64)] +Aggregate Attributes [1]: [max(csales#64)#67] +Results [1]: [max(csales#64)#67 AS tpcds_cmax#68] -Subquery:4 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#52 IN dynamicpruning#53 -BroadcastExchange (107) -+- * ColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan parquet spark_catalog.default.date_dim (103) +Subquery:4 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#55 IN dynamicpruning#56 +BroadcastExchange (111) ++- * ColumnarToRow (110) + +- CometProject (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#55, d_year#63] +Output [2]: [d_date_sk#58, d_year#69] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(104) CometFilter -Input [2]: [d_date_sk#55, d_year#63] -Condition : (d_year#63 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#55)) +(108) CometFilter +Input [2]: [d_date_sk#58, d_year#69] +Condition : (d_year#69 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#58)) -(105) CometProject -Input [2]: [d_date_sk#55, d_year#63] -Arguments: [d_date_sk#55], [d_date_sk#55] +(109) CometProject +Input [2]: [d_date_sk#58, d_year#69] +Arguments: [d_date_sk#58], [d_date_sk#58] -(106) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#55] +(110) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#58] -(107) BroadcastExchange -Input [1]: [d_date_sk#55] +(111) BroadcastExchange +Input [1]: [d_date_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] +Subquery:6 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index 51a411e3c9..e49c73087f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -1,154 +1,170 @@ -WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] +WholeStageCodegen (24) + HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar - WholeStageCodegen (19) + WholeStageCodegen (23) HashAggregate [sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (9) + WholeStageCodegen (11) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,c_customer_sk] - CometSort [cs_bill_customer_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - RowToColumnar - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [item_sk] - CometFilter [cnt] - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow + Project [cs_quantity,cs_list_price,cs_sold_date_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometSort [cs_bill_customer_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + RowToColumnar + WholeStageCodegen (5) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [ssales] + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + InputAdapter + WholeStageCodegen (9) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] Subquery #3 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [max] + WholeStageCodegen (5) + HashAggregate [max] [max(csales),tpcds_cmax,max] + ColumnarToRow + InputAdapter CometColumnarExchange #10 - CometHashAggregate [csales] - CometHashAggregate [c_customer_sk,sum,isEmpty] - CometColumnarExchange [c_customer_sk] #11 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [c_customer_sk] #9 - InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [c_customer_sk,sum,isEmpty] - CometColumnarExchange [c_customer_sk] #8 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) + RowToColumnar + WholeStageCodegen (4) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] ColumnarToRow InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] + CometColumnarExchange [c_customer_sk] #11 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [c_customer_sk] #9 + InputAdapter + ReusedExchange [d_date_sk] #12 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_sk] #8 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (18) + WholeStageCodegen (22) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,c_customer_sk] - CometSort [ws_bill_customer_sk] - CometColumnarExchange [ws_bill_customer_sk] #13 - RowToColumnar - WholeStageCodegen (14) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [ssales] + Project [ws_quantity,ws_list_price,ws_sold_date_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (17) + ColumnarToRow + InputAdapter + CometSort [ws_bill_customer_sk] + CometColumnarExchange [ws_bill_customer_sk] #13 + RowToColumnar + WholeStageCodegen (16) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + InputAdapter + WholeStageCodegen (20) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,sum,isEmpty] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 5d9f6a6715..3c3ac56e1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -1,101 +1,105 @@ == Physical Plan == -* ColumnarToRow (97) -+- CometTakeOrderedAndProject (96) - +- CometUnion (95) - :- CometHashAggregate (68) - : +- CometColumnarExchange (67) - : +- RowToColumnar (66) - : +- * HashAggregate (65) - : +- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * ColumnarToRow (47) - : : : +- CometSortMergeJoin (46) - : : : :- CometSort (28) - : : : : +- CometColumnarExchange (27) - : : : : +- RowToColumnar (26) - : : : : +- * Project (25) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (24) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (23) - : : : : +- * ColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometHashAggregate (19) - : : : : +- CometColumnarExchange (18) - : : : : +- RowToColumnar (17) - : : : : +- * HashAggregate (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (7) - : : : : +- BroadcastExchange (13) - : : : : +- * ColumnarToRow (12) - : : : : +- CometFilter (11) - : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : +- CometSort (45) - : : : +- CometProject (44) - : : : +- CometFilter (43) - : : : +- CometHashAggregate (42) - : : : +- CometColumnarExchange (41) - : : : +- RowToColumnar (40) - : : : +- * HashAggregate (39) - : : : +- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * ColumnarToRow (32) - : : : : +- CometProject (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.store_sales (29) - : : : +- BroadcastExchange (36) - : : : +- * ColumnarToRow (35) - : : : +- CometFilter (34) - : : : +- CometScan parquet spark_catalog.default.customer (33) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometSortMergeJoin (57) - : : :- CometSort (51) - : : : +- CometColumnarExchange (50) - : : : +- CometFilter (49) - : : : +- CometScan parquet spark_catalog.default.customer (48) - : : +- CometSort (56) - : : +- CometProject (55) - : : +- CometFilter (54) - : : +- CometHashAggregate (53) - : : +- ReusedExchange (52) - : +- ReusedExchange (62) - +- CometHashAggregate (94) - +- CometColumnarExchange (93) - +- RowToColumnar (92) - +- * HashAggregate (91) - +- * Project (90) - +- * BroadcastHashJoin Inner BuildRight (89) - :- * Project (87) - : +- * BroadcastHashJoin Inner BuildRight (86) - : :- * ColumnarToRow (84) - : : +- CometSortMergeJoin (83) - : : :- CometSort (77) - : : : +- CometColumnarExchange (76) - : : : +- RowToColumnar (75) - : : : +- * Project (74) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (73) - : : : :- * ColumnarToRow (71) - : : : : +- CometFilter (70) - : : : : +- CometScan parquet spark_catalog.default.web_sales (69) - : : : +- ReusedExchange (72) - : : +- CometSort (82) - : : +- CometProject (81) - : : +- CometFilter (80) - : : +- CometHashAggregate (79) - : : +- ReusedExchange (78) - : +- ReusedExchange (85) - +- ReusedExchange (88) +TakeOrderedAndProject (101) ++- Union (100) + :- * HashAggregate (71) + : +- * ColumnarToRow (70) + : +- CometColumnarExchange (69) + : +- RowToColumnar (68) + : +- * HashAggregate (67) + : +- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * SortMergeJoin LeftSemi (48) + : : : :- * ColumnarToRow (29) + : : : : +- CometSort (28) + : : : : +- CometColumnarExchange (27) + : : : : +- RowToColumnar (26) + : : : : +- * Project (25) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (24) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (23) + : : : : +- * Project (22) + : : : : +- * Filter (21) + : : : : +- * HashAggregate (20) + : : : : +- * ColumnarToRow (19) + : : : : +- CometColumnarExchange (18) + : : : : +- RowToColumnar (17) + : : : : +- * HashAggregate (16) + : : : : +- * Project (15) + : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : :- * Project (9) + : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : :- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : : +- ReusedExchange (7) + : : : : +- BroadcastExchange (13) + : : : : +- * ColumnarToRow (12) + : : : : +- CometFilter (11) + : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : +- * Sort (47) + : : : +- * Project (46) + : : : +- * Filter (45) + : : : +- * HashAggregate (44) + : : : +- * ColumnarToRow (43) + : : : +- CometColumnarExchange (42) + : : : +- RowToColumnar (41) + : : : +- * HashAggregate (40) + : : : +- * Project (39) + : : : +- * BroadcastHashJoin Inner BuildRight (38) + : : : :- * ColumnarToRow (33) + : : : : +- CometProject (32) + : : : : +- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.store_sales (30) + : : : +- BroadcastExchange (37) + : : : +- * ColumnarToRow (36) + : : : +- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- BroadcastExchange (61) + : : +- * SortMergeJoin LeftSemi (60) + : : :- * ColumnarToRow (53) + : : : +- CometSort (52) + : : : +- CometColumnarExchange (51) + : : : +- CometFilter (50) + : : : +- CometScan parquet spark_catalog.default.customer (49) + : : +- * Sort (59) + : : +- * Project (58) + : : +- * Filter (57) + : : +- * HashAggregate (56) + : : +- * ColumnarToRow (55) + : : +- ReusedExchange (54) + : +- ReusedExchange (64) + +- * HashAggregate (99) + +- * ColumnarToRow (98) + +- CometColumnarExchange (97) + +- RowToColumnar (96) + +- * HashAggregate (95) + +- * Project (94) + +- * BroadcastHashJoin Inner BuildRight (93) + :- * Project (91) + : +- * BroadcastHashJoin Inner BuildRight (90) + : :- * SortMergeJoin LeftSemi (88) + : : :- * ColumnarToRow (81) + : : : +- CometSort (80) + : : : +- CometColumnarExchange (79) + : : : +- RowToColumnar (78) + : : : +- * Project (77) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (76) + : : : :- * ColumnarToRow (74) + : : : : +- CometFilter (73) + : : : : +- CometScan parquet spark_catalog.default.web_sales (72) + : : : +- ReusedExchange (75) + : : +- * Sort (87) + : : +- * Project (86) + : : +- * Filter (85) + : : +- * HashAggregate (84) + : : +- * ColumnarToRow (83) + : : +- ReusedExchange (82) + : +- ReusedExchange (89) + +- ReusedExchange (92) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -128,7 +132,7 @@ Condition : isnotnull(ss_item_sk#7) (6) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(7) ReusedExchange [Reuses operator id: 107] +(7) ReusedExchange [Reuses operator id: 111] Output [2]: [d_date_sk#10, d_date#11] (8) BroadcastHashJoin [codegen id : 3] @@ -183,29 +187,31 @@ Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) CometHashAggregate +(19) ColumnarToRow [codegen id : 4] +Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] + +(20) HashAggregate [codegen id : 4] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#17] +Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(20) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(21) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] +(21) Filter [codegen id : 4] +Input [2]: [item_sk#18, cnt#19] +Condition : (cnt#19 > 4) -(22) ColumnarToRow [codegen id : 4] -Input [1]: [item_sk#17] +(22) Project [codegen id : 4] +Output [1]: [item_sk#18] +Input [2]: [item_sk#18, cnt#19] (23) BroadcastExchange -Input [1]: [item_sk#17] +Input [1]: [item_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] (24) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#17] +Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None @@ -224,500 +230,537 @@ Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, Come Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] +(29) ColumnarToRow [codegen id : 6] +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] +Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) +(31) CometFilter +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#20) -(31) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] +(32) CometProject +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -(32) ColumnarToRow [codegen id : 7] -Input [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] +(33) ColumnarToRow [codegen id : 8] +Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] (unknown) Scan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] +Output [1]: [c_customer_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(34) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) +(35) CometFilter +Input [1]: [c_customer_sk#24] +Condition : isnotnull(c_customer_sk#24) -(35) ColumnarToRow [codegen id : 6] -Input [1]: [c_customer_sk#23] +(36) ColumnarToRow [codegen id : 7] +Input [1]: [c_customer_sk#24] -(36) BroadcastExchange -Input [1]: [c_customer_sk#23] +(37) BroadcastExchange +Input [1]: [c_customer_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_customer_sk#19] -Right keys [1]: [c_customer_sk#23] +(38) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_customer_sk#20] +Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(38) Project [codegen id : 7] -Output [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] +(39) Project [codegen id : 8] +Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -(39) HashAggregate [codegen id : 7] -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] -Aggregate Attributes [2]: [sum#24, isEmpty#25] -Results [3]: [c_customer_sk#23, sum#26, isEmpty#27] +(40) HashAggregate [codegen id : 8] +Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Keys [1]: [c_customer_sk#24] +Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [2]: [sum#25, isEmpty#26] +Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(40) RowToColumnar -Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] +(41) RowToColumnar +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(41) CometColumnarExchange -Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(42) CometColumnarExchange +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(42) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] +(43) ColumnarToRow [codegen id : 9] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(43) CometFilter -Input [2]: [c_customer_sk#23, ssales#28] -Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#29, [id=#30]))) +(44) HashAggregate [codegen id : 9] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Keys [1]: [c_customer_sk#24] +Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(44) CometProject -Input [2]: [c_customer_sk#23, ssales#28] -Arguments: [c_customer_sk#23], [c_customer_sk#23] +(45) Filter [codegen id : 9] +Input [2]: [c_customer_sk#24, ssales#30] +Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) -(45) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] +(46) Project [codegen id : 9] +Output [1]: [c_customer_sk#24] +Input [2]: [c_customer_sk#24, ssales#30] -(46) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi +(47) Sort [codegen id : 9] +Input [1]: [c_customer_sk#24] +Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(47) ColumnarToRow [codegen id : 12] -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +(48) SortMergeJoin [codegen id : 16] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#24] +Join type: LeftSemi +Join condition: None (unknown) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Output [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(49) CometFilter -Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] -Condition : isnotnull(c_customer_sk#31) +(50) CometFilter +Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +Condition : isnotnull(c_customer_sk#33) -(50) CometColumnarExchange -Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] -Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(51) CometColumnarExchange +Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +Arguments: hashpartitioning(c_customer_sk#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(51) CometSort -Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] -Arguments: [c_customer_sk#31, c_first_name#32, c_last_name#33], [c_customer_sk#31 ASC NULLS FIRST] +(52) CometSort +Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +Arguments: [c_customer_sk#33, c_first_name#34, c_last_name#35], [c_customer_sk#33 ASC NULLS FIRST] -(52) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#23, sum#26, isEmpty#27] +(53) ColumnarToRow [codegen id : 10] +Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -(53) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] +(54) ReusedExchange [Reuses operator id: 42] +Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(54) CometFilter -Input [2]: [c_customer_sk#23, ssales#28] -Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30]))) +(55) ColumnarToRow [codegen id : 13] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(55) CometProject -Input [2]: [c_customer_sk#23, ssales#28] -Arguments: [c_customer_sk#23], [c_customer_sk#23] +(56) HashAggregate [codegen id : 13] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Keys [1]: [c_customer_sk#24] +Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(56) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] +(57) Filter [codegen id : 13] +Input [2]: [c_customer_sk#24, ssales#30] +Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(57) CometSortMergeJoin -Left output [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] -Right output [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#31], [c_customer_sk#23], LeftSemi +(58) Project [codegen id : 13] +Output [1]: [c_customer_sk#24] +Input [2]: [c_customer_sk#24, ssales#30] -(58) ColumnarToRow [codegen id : 10] -Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +(59) Sort [codegen id : 13] +Input [1]: [c_customer_sk#24] +Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 + +(60) SortMergeJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#33] +Right keys [1]: [c_customer_sk#24] +Join type: LeftSemi +Join condition: None -(59) BroadcastExchange -Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +(61) BroadcastExchange +Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 12] +(62) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#31] +Right keys [1]: [c_customer_sk#33] Join type: Inner Join condition: None -(61) Project [codegen id : 12] -Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#31, c_first_name#32, c_last_name#33] +(63) Project [codegen id : 16] +Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35] +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#33, c_first_name#34, c_last_name#35] -(62) ReusedExchange [Reuses operator id: 102] -Output [1]: [d_date_sk#34] +(64) ReusedExchange [Reuses operator id: 106] +Output [1]: [d_date_sk#36] -(63) BroadcastHashJoin [codegen id : 12] +(65) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#34] +Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(64) Project [codegen id : 12] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33, d_date_sk#34] +(66) Project [codegen id : 16] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35, d_date_sk#36] -(65) HashAggregate [codegen id : 12] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] -Keys [2]: [c_last_name#33, c_first_name#32] +(67) HashAggregate [codegen id : 16] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] +Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] +Aggregate Attributes [2]: [sum#37, isEmpty#38] +Results [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] + +(68) RowToColumnar +Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -(66) RowToColumnar -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] +(69) CometColumnarExchange +Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] +Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(67) CometColumnarExchange -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Arguments: hashpartitioning(c_last_name#33, c_first_name#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(70) ColumnarToRow [codegen id : 17] +Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -(68) CometHashAggregate -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Keys [2]: [c_last_name#33, c_first_name#32] +(71) HashAggregate [codegen id : 17] +Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] +Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] +Aggregate Attributes [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41] +Results [3]: [c_last_name#35, c_first_name#34, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41 AS sales#42] (unknown) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] +Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#43), dynamicpruningexpression(ws_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(70) CometFilter -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Condition : isnotnull(ws_bill_customer_sk#40) +(73) CometFilter +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Condition : isnotnull(ws_bill_customer_sk#44) -(71) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] +(74) ColumnarToRow [codegen id : 22] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(72) ReusedExchange [Reuses operator id: 23] -Output [1]: [item_sk#17] +(75) ReusedExchange [Reuses operator id: 23] +Output [1]: [item_sk#18] -(73) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [item_sk#17] +(76) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_item_sk#43] +Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(74) Project [codegen id : 17] -Output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] +(77) Project [codegen id : 22] +Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] + +(78) RowToColumnar +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(75) RowToColumnar -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] +(79) CometColumnarExchange +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(76) CometColumnarExchange -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: hashpartitioning(ws_bill_customer_sk#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(80) CometSort +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47], [ws_bill_customer_sk#44 ASC NULLS FIRST] -(77) CometSort -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40 ASC NULLS FIRST] +(81) ColumnarToRow [codegen id : 23] +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(78) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#23, sum#26, isEmpty#27] +(82) ReusedExchange [Reuses operator id: 42] +Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(79) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] +(83) ColumnarToRow [codegen id : 26] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(80) CometFilter -Input [2]: [c_customer_sk#23, ssales#28] -Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30]))) +(84) HashAggregate [codegen id : 26] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Keys [1]: [c_customer_sk#24] +Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(81) CometProject -Input [2]: [c_customer_sk#23, ssales#28] -Arguments: [c_customer_sk#23], [c_customer_sk#23] +(85) Filter [codegen id : 26] +Input [2]: [c_customer_sk#24, ssales#30] +Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(82) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] +(86) Project [codegen id : 26] +Output [1]: [c_customer_sk#24] +Input [2]: [c_customer_sk#24, ssales#30] -(83) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [c_customer_sk#23] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#23], LeftSemi +(87) Sort [codegen id : 26] +Input [1]: [c_customer_sk#24] +Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(84) ColumnarToRow [codegen id : 24] -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] +(88) SortMergeJoin [codegen id : 33] +Left keys [1]: [ws_bill_customer_sk#44] +Right keys [1]: [c_customer_sk#24] +Join type: LeftSemi +Join condition: None -(85) ReusedExchange [Reuses operator id: 59] -Output [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] +(89) ReusedExchange [Reuses operator id: 61] +Output [3]: [c_customer_sk#49, c_first_name#50, c_last_name#51] -(86) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ws_bill_customer_sk#40] -Right keys [1]: [c_customer_sk#45] +(90) BroadcastHashJoin [codegen id : 33] +Left keys [1]: [ws_bill_customer_sk#44] +Right keys [1]: [c_customer_sk#49] Join type: Inner Join condition: None -(87) Project [codegen id : 24] -Output [5]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47] -Input [7]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_customer_sk#45, c_first_name#46, c_last_name#47] +(91) Project [codegen id : 33] +Output [5]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#50, c_last_name#51] +Input [7]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_customer_sk#49, c_first_name#50, c_last_name#51] -(88) ReusedExchange [Reuses operator id: 102] -Output [1]: [d_date_sk#48] +(92) ReusedExchange [Reuses operator id: 106] +Output [1]: [d_date_sk#52] -(89) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ws_sold_date_sk#43] -Right keys [1]: [d_date_sk#48] +(93) BroadcastHashJoin [codegen id : 33] +Left keys [1]: [ws_sold_date_sk#47] +Right keys [1]: [d_date_sk#52] Join type: Inner Join condition: None -(90) Project [codegen id : 24] -Output [4]: [ws_quantity#41, ws_list_price#42, c_first_name#46, c_last_name#47] -Input [6]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47, d_date_sk#48] +(94) Project [codegen id : 33] +Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] +Input [6]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#50, c_last_name#51, d_date_sk#52] -(91) HashAggregate [codegen id : 24] -Input [4]: [ws_quantity#41, ws_list_price#42, c_first_name#46, c_last_name#47] -Keys [2]: [c_last_name#47, c_first_name#46] -Functions [1]: [partial_sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [4]: [c_last_name#47, c_first_name#46, sum#51, isEmpty#52] +(95) HashAggregate [codegen id : 33] +Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] +Keys [2]: [c_last_name#51, c_first_name#50] +Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] +Aggregate Attributes [2]: [sum#53, isEmpty#54] +Results [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] -(92) RowToColumnar -Input [4]: [c_last_name#47, c_first_name#46, sum#51, isEmpty#52] +(96) RowToColumnar +Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] -(93) CometColumnarExchange -Input [4]: [c_last_name#47, c_first_name#46, sum#51, isEmpty#52] -Arguments: hashpartitioning(c_last_name#47, c_first_name#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(97) CometColumnarExchange +Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] +Arguments: hashpartitioning(c_last_name#51, c_first_name#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(94) CometHashAggregate -Input [4]: [c_last_name#47, c_first_name#46, sum#51, isEmpty#52] -Keys [2]: [c_last_name#47, c_first_name#46] -Functions [1]: [sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] +(98) ColumnarToRow [codegen id : 34] +Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] -(95) CometUnion -Child 0 Input [3]: [c_last_name#33, c_first_name#32, sales#53] -Child 1 Input [3]: [c_last_name#47, c_first_name#46, sales#54] +(99) HashAggregate [codegen id : 34] +Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] +Keys [2]: [c_last_name#51, c_first_name#50] +Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] +Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#57] +Results [3]: [c_last_name#51, c_first_name#50, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#57 AS sales#58] -(96) CometTakeOrderedAndProject -Input [3]: [c_last_name#33, c_first_name#32, sales#53] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#33 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,sales#53 ASC NULLS FIRST], output=[c_last_name#33,c_first_name#32,sales#53]), 100, [c_last_name#33 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, sales#53 ASC NULLS FIRST], [c_last_name#33, c_first_name#32, sales#53] +(100) Union -(97) ColumnarToRow [codegen id : 25] -Input [3]: [c_last_name#33, c_first_name#32, sales#53] +(101) TakeOrderedAndProject +Input [3]: [c_last_name#35, c_first_name#34, sales#42] +Arguments: 100, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, sales#42 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, sales#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (102) -+- * ColumnarToRow (101) - +- CometProject (100) - +- CometFilter (99) - +- CometScan parquet spark_catalog.default.date_dim (98) +BroadcastExchange (106) ++- * ColumnarToRow (105) + +- CometProject (104) + +- CometFilter (103) + +- CometScan parquet spark_catalog.default.date_dim (102) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#55, d_moy#56] +Output [3]: [d_date_sk#36, d_year#59, d_moy#60] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(99) CometFilter -Input [3]: [d_date_sk#34, d_year#55, d_moy#56] -Condition : ((((isnotnull(d_year#55) AND isnotnull(d_moy#56)) AND (d_year#55 = 2000)) AND (d_moy#56 = 2)) AND isnotnull(d_date_sk#34)) +(103) CometFilter +Input [3]: [d_date_sk#36, d_year#59, d_moy#60] +Condition : ((((isnotnull(d_year#59) AND isnotnull(d_moy#60)) AND (d_year#59 = 2000)) AND (d_moy#60 = 2)) AND isnotnull(d_date_sk#36)) -(100) CometProject -Input [3]: [d_date_sk#34, d_year#55, d_moy#56] -Arguments: [d_date_sk#34], [d_date_sk#34] +(104) CometProject +Input [3]: [d_date_sk#36, d_year#59, d_moy#60] +Arguments: [d_date_sk#36], [d_date_sk#36] -(101) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#34] +(105) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#36] -(102) BroadcastExchange -Input [1]: [d_date_sk#34] +(106) BroadcastExchange +Input [1]: [d_date_sk#36] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (107) -+- * ColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan parquet spark_catalog.default.date_dim (103) +BroadcastExchange (111) ++- * ColumnarToRow (110) + +- CometProject (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#57] +Output [3]: [d_date_sk#10, d_date#11, d_year#61] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(104) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#57] -Condition : (d_year#57 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +(108) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#61] +Condition : (d_year#61 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(105) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#57] +(109) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#61] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(106) ColumnarToRow [codegen id : 1] +(110) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(107) BroadcastExchange +(111) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#29, [id=#30] -* ColumnarToRow (124) -+- CometHashAggregate (123) - +- CometColumnarExchange (122) - +- CometHashAggregate (121) - +- CometHashAggregate (120) - +- CometColumnarExchange (119) - +- RowToColumnar (118) - +- * HashAggregate (117) - +- * Project (116) - +- * BroadcastHashJoin Inner BuildRight (115) - :- * Project (113) - : +- * BroadcastHashJoin Inner BuildRight (112) - : :- * ColumnarToRow (110) - : : +- CometFilter (109) - : : +- CometScan parquet spark_catalog.default.store_sales (108) - : +- ReusedExchange (111) - +- ReusedExchange (114) +Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +* HashAggregate (130) ++- * ColumnarToRow (129) + +- CometColumnarExchange (128) + +- RowToColumnar (127) + +- * HashAggregate (126) + +- * HashAggregate (125) + +- * ColumnarToRow (124) + +- CometColumnarExchange (123) + +- RowToColumnar (122) + +- * HashAggregate (121) + +- * Project (120) + +- * BroadcastHashJoin Inner BuildRight (119) + :- * Project (117) + : +- * BroadcastHashJoin Inner BuildRight (116) + : :- * ColumnarToRow (114) + : : +- CometFilter (113) + : : +- CometScan parquet spark_catalog.default.store_sales (112) + : +- ReusedExchange (115) + +- ReusedExchange (118) (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61] +Output [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#61), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#62)] +PartitionFilters: [isnotnull(ss_sold_date_sk#65), dynamicpruningexpression(ss_sold_date_sk#65 IN dynamicpruning#66)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(109) CometFilter -Input [4]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61] -Condition : isnotnull(ss_customer_sk#58) +(113) CometFilter +Input [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] +Condition : isnotnull(ss_customer_sk#62) -(110) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61] +(114) ColumnarToRow [codegen id : 3] +Input [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] -(111) ReusedExchange [Reuses operator id: 36] -Output [1]: [c_customer_sk#63] +(115) ReusedExchange [Reuses operator id: 37] +Output [1]: [c_customer_sk#67] -(112) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#58] -Right keys [1]: [c_customer_sk#63] +(116) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_customer_sk#62] +Right keys [1]: [c_customer_sk#67] Join type: Inner Join condition: None -(113) Project [codegen id : 3] -Output [4]: [ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61, c_customer_sk#63] -Input [5]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61, c_customer_sk#63] +(117) Project [codegen id : 3] +Output [4]: [ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67] +Input [5]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67] -(114) ReusedExchange [Reuses operator id: 129] -Output [1]: [d_date_sk#64] +(118) ReusedExchange [Reuses operator id: 135] +Output [1]: [d_date_sk#68] -(115) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#61] -Right keys [1]: [d_date_sk#64] +(119) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#65] +Right keys [1]: [d_date_sk#68] Join type: Inner Join condition: None -(116) Project [codegen id : 3] -Output [3]: [ss_quantity#59, ss_sales_price#60, c_customer_sk#63] -Input [5]: [ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61, c_customer_sk#63, d_date_sk#64] +(120) Project [codegen id : 3] +Output [3]: [ss_quantity#63, ss_sales_price#64, c_customer_sk#67] +Input [5]: [ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67, d_date_sk#68] -(117) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#59, ss_sales_price#60, c_customer_sk#63] -Keys [1]: [c_customer_sk#63] -Functions [1]: [partial_sum((cast(ss_quantity#59 as decimal(10,0)) * ss_sales_price#60))] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [3]: [c_customer_sk#63, sum#67, isEmpty#68] +(121) HashAggregate [codegen id : 3] +Input [3]: [ss_quantity#63, ss_sales_price#64, c_customer_sk#67] +Keys [1]: [c_customer_sk#67] +Functions [1]: [partial_sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))] +Aggregate Attributes [2]: [sum#69, isEmpty#70] +Results [3]: [c_customer_sk#67, sum#71, isEmpty#72] -(118) RowToColumnar -Input [3]: [c_customer_sk#63, sum#67, isEmpty#68] +(122) RowToColumnar +Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] -(119) CometColumnarExchange -Input [3]: [c_customer_sk#63, sum#67, isEmpty#68] -Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(123) CometColumnarExchange +Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] +Arguments: hashpartitioning(c_customer_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(120) CometHashAggregate -Input [3]: [c_customer_sk#63, sum#67, isEmpty#68] -Keys [1]: [c_customer_sk#63] -Functions [1]: [sum((cast(ss_quantity#59 as decimal(10,0)) * ss_sales_price#60))] +(124) ColumnarToRow [codegen id : 4] +Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] -(121) CometHashAggregate -Input [1]: [csales#69] +(125) HashAggregate [codegen id : 4] +Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] +Keys [1]: [c_customer_sk#67] +Functions [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))#73] +Results [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))#73 AS csales#74] + +(126) HashAggregate [codegen id : 4] +Input [1]: [csales#74] Keys: [] -Functions [1]: [partial_max(csales#69)] +Functions [1]: [partial_max(csales#74)] +Aggregate Attributes [1]: [max#75] +Results [1]: [max#76] + +(127) RowToColumnar +Input [1]: [max#76] -(122) CometColumnarExchange -Input [1]: [max#70] +(128) CometColumnarExchange +Input [1]: [max#76] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(123) CometHashAggregate -Input [1]: [max#70] -Keys: [] -Functions [1]: [max(csales#69)] +(129) ColumnarToRow [codegen id : 5] +Input [1]: [max#76] -(124) ColumnarToRow [codegen id : 4] -Input [1]: [tpcds_cmax#71] +(130) HashAggregate [codegen id : 5] +Input [1]: [max#76] +Keys: [] +Functions [1]: [max(csales#74)] +Aggregate Attributes [1]: [max(csales#74)#77] +Results [1]: [max(csales#74)#77 AS tpcds_cmax#78] -Subquery:4 Hosting operator id = 108 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#62 -BroadcastExchange (129) -+- * ColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan parquet spark_catalog.default.date_dim (125) +Subquery:4 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#65 IN dynamicpruning#66 +BroadcastExchange (135) ++- * ColumnarToRow (134) + +- CometProject (133) + +- CometFilter (132) + +- CometScan parquet spark_catalog.default.date_dim (131) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_year#72] +Output [2]: [d_date_sk#68, d_year#79] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(126) CometFilter -Input [2]: [d_date_sk#64, d_year#72] -Condition : (d_year#72 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#64)) +(132) CometFilter +Input [2]: [d_date_sk#68, d_year#79] +Condition : (d_year#79 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#68)) -(127) CometProject -Input [2]: [d_date_sk#64, d_year#72] -Arguments: [d_date_sk#64], [d_date_sk#64] +(133) CometProject +Input [2]: [d_date_sk#68, d_year#79] +Arguments: [d_date_sk#68], [d_date_sk#68] -(128) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#64] +(134) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#68] -(129) BroadcastExchange -Input [1]: [d_date_sk#64] +(135) BroadcastExchange +Input [1]: [d_date_sk#68] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] +Subquery:5 Hosting operator id = 57 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] -Subquery:6 Hosting operator id = 69 Hosting Expression = ws_sold_date_sk#43 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 72 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 80 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] +Subquery:7 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 2dd7ddf4af..83cea75a65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -1,178 +1,203 @@ -WholeStageCodegen (25) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion - CometHashAggregate [c_last_name,c_first_name,sum,isEmpty] +TakeOrderedAndProject [c_last_name,c_first_name,sales] + Union + WholeStageCodegen (17) + HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [c_last_name,c_first_name] #1 RowToColumnar - WholeStageCodegen (12) + WholeStageCodegen (16) HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_list_price,c_first_name,c_last_name] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - ColumnarToRow + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter - CometSortMergeJoin [cs_bill_customer_sk,c_customer_sk] - CometSort [cs_bill_customer_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - RowToColumnar - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometSort [cs_bill_customer_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + RowToColumnar + WholeStageCodegen (5) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + InputAdapter + WholeStageCodegen (9) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + Subquery #3 + WholeStageCodegen (5) + HashAggregate [max] [max(csales),tpcds_cmax,max] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [item_sk] - CometFilter [cnt] - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) + CometColumnarExchange [c_customer_sk] #11 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [ssales] - Subquery #3 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [max] - CometColumnarExchange #10 - CometHashAggregate [csales] - CometHashAggregate [c_customer_sk,sum,isEmpty] - CometColumnarExchange [c_customer_sk] #11 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [c_customer_sk] #9 - InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [c_customer_sk,sum,isEmpty] - CometColumnarExchange [c_customer_sk] #8 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [c_customer_sk] #9 + InputAdapter + ReusedExchange [d_date_sk] #12 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_sk] #8 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter BroadcastExchange #13 - WholeStageCodegen (10) - ColumnarToRow + WholeStageCodegen (14) + SortMergeJoin [c_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometSort [c_customer_sk] + CometColumnarExchange [c_customer_sk] #14 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - CometSortMergeJoin [c_customer_sk,c_customer_sk] - CometSort [c_customer_sk] - CometColumnarExchange [c_customer_sk] #14 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [ssales] + WholeStageCodegen (13) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,sum,isEmpty] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter ReusedExchange [d_date_sk] #3 - CometHashAggregate [c_last_name,c_first_name,sum,isEmpty] + WholeStageCodegen (34) + HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [c_last_name,c_first_name] #15 RowToColumnar - WholeStageCodegen (24) + WholeStageCodegen (33) HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_list_price,c_first_name,c_last_name] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - ColumnarToRow + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - CometSortMergeJoin [ws_bill_customer_sk,c_customer_sk] - CometSort [ws_bill_customer_sk] - CometColumnarExchange [ws_bill_customer_sk] #16 - RowToColumnar - WholeStageCodegen (17) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [ssales] + WholeStageCodegen (23) + ColumnarToRow + InputAdapter + CometSort [ws_bill_customer_sk] + CometColumnarExchange [ws_bill_customer_sk] #16 + RowToColumnar + WholeStageCodegen (22) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + InputAdapter + WholeStageCodegen (26) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,sum,isEmpty] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index a7ccfef3ab..51943ef8bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -1,51 +1,53 @@ == Physical Plan == -* ColumnarToRow (47) -+- CometFilter (46) - +- CometHashAggregate (45) - +- CometColumnarExchange (44) - +- CometHashAggregate (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- RowToColumnar (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometColumnarExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometColumnarExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.item (21) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_address (33) +* Filter (49) ++- * HashAggregate (48) + +- * ColumnarToRow (47) + +- CometColumnarExchange (46) + +- RowToColumnar (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- * ColumnarToRow (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * ColumnarToRow (13) + : : : : +- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.store (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.item (21) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_address (33) (unknown) Scan parquet spark_catalog.default.store_sales @@ -236,103 +238,117 @@ Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_st Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(42) CometHashAggregate +(42) ColumnarToRow [codegen id : 6] +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] + +(43) HashAggregate [codegen id : 6] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] +Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(43) CometHashAggregate -Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#30] +(44) HashAggregate [codegen id : 6] +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] -Functions [1]: [partial_sum(netpaid#30)] +Functions [1]: [partial_sum(netpaid#31)] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] + +(45) RowToColumnar +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(44) CometColumnarExchange -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#31, isEmpty#32] +(46) CometColumnarExchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(45) CometHashAggregate -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#31, isEmpty#32] -Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] -Functions [1]: [sum(netpaid#30)] +(47) ColumnarToRow [codegen id : 7] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(46) CometFilter -Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#33] -Condition : (isnotnull(paid#33) AND (cast(paid#33 as decimal(33,8)) > cast(Subquery scalar-subquery#34, [id=#35] as decimal(33,8)))) +(48) HashAggregate [codegen id : 7] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] +Functions [1]: [sum(netpaid#31)] +Aggregate Attributes [1]: [sum(netpaid#31)#36] +Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(47) ColumnarToRow [codegen id : 6] -Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#33] +(49) Filter [codegen id : 7] +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] +Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#34, [id=#35] -* ColumnarToRow (77) -+- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- CometHashAggregate (74) - +- CometHashAggregate (73) - +- CometColumnarExchange (72) - +- RowToColumnar (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : :- * ColumnarToRow (54) - : : : : +- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (55) - : : +- BroadcastExchange (61) - : : +- * ColumnarToRow (60) - : : +- CometFilter (59) - : : +- CometScan parquet spark_catalog.default.item (58) - : +- ReusedExchange (64) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (81) ++- * ColumnarToRow (80) + +- CometColumnarExchange (79) + +- RowToColumnar (78) + +- * HashAggregate (77) + +- * HashAggregate (76) + +- * ColumnarToRow (75) + +- CometColumnarExchange (74) + +- RowToColumnar (73) + +- * HashAggregate (72) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * Project (65) + : : +- * BroadcastHashJoin Inner BuildRight (64) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * ColumnarToRow (56) + : : : : +- CometProject (55) + : : : : +- CometSortMergeJoin (54) + : : : : :- CometSort (51) + : : : : : +- ReusedExchange (50) + : : : : +- CometSort (53) + : : : : +- ReusedExchange (52) + : : : +- ReusedExchange (57) + : : +- BroadcastExchange (63) + : : +- * ColumnarToRow (62) + : : +- CometFilter (61) + : : +- CometScan parquet spark_catalog.default.item (60) + : +- ReusedExchange (66) + +- ReusedExchange (69) + + +(50) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(49) CometSort +(51) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(50) ReusedExchange [Reuses operator id: 9] +(52) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(51) CometSort +(53) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(52) CometSortMergeJoin +(54) CometSortMergeJoin Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Right output [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(53) CometProject +(55) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(54) ColumnarToRow [codegen id : 5] +(56) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(55) ReusedExchange [Reuses operator id: 18] +(57) ReusedExchange [Reuses operator id: 18] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(56) BroadcastHashJoin [codegen id : 5] +(58) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(57) Project [codegen id : 5] +(59) Project [codegen id : 5] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -343,87 +359,99 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) CometFilter +(61) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : isnotnull(i_item_sk#15) -(60) ColumnarToRow [codegen id : 2] +(62) ColumnarToRow [codegen id : 2] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(61) BroadcastExchange +(63) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(62) BroadcastHashJoin [codegen id : 5] +(64) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(63) Project [codegen id : 5] +(65) Project [codegen id : 5] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(64) ReusedExchange [Reuses operator id: 30] +(66) ReusedExchange [Reuses operator id: 30] Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(65) BroadcastHashJoin [codegen id : 5] +(67) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(66) Project [codegen id : 5] +(68) Project [codegen id : 5] Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(67) ReusedExchange [Reuses operator id: 36] +(69) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#25, ca_zip#26, ca_country#27] -(68) BroadcastHashJoin [codegen id : 5] +(70) BroadcastHashJoin [codegen id : 5] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(69) Project [codegen id : 5] +(71) Project [codegen id : 5] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(70) HashAggregate [codegen id : 5] +(72) HashAggregate [codegen id : 5] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#36] -Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] +Aggregate Attributes [1]: [sum#40] +Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] -(71) RowToColumnar -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] +(73) RowToColumnar +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] -(72) CometColumnarExchange -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] +(74) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(73) CometHashAggregate -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] +(75) ColumnarToRow [codegen id : 6] +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] + +(76) HashAggregate [codegen id : 6] +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(74) CometHashAggregate -Input [1]: [netpaid#30] +(77) HashAggregate [codegen id : 6] +Input [1]: [netpaid#31] Keys: [] -Functions [1]: [partial_avg(netpaid#30)] +Functions [1]: [partial_avg(netpaid#31)] +Aggregate Attributes [2]: [sum#42, count#43] +Results [2]: [sum#44, count#45] + +(78) RowToColumnar +Input [2]: [sum#44, count#45] -(75) CometColumnarExchange -Input [2]: [sum#38, count#39] +(79) CometColumnarExchange +Input [2]: [sum#44, count#45] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(76) CometHashAggregate -Input [2]: [sum#38, count#39] -Keys: [] -Functions [1]: [avg(netpaid#30)] +(80) ColumnarToRow [codegen id : 7] +Input [2]: [sum#44, count#45] -(77) ColumnarToRow [codegen id : 6] -Input [1]: [(0.05 * avg(netpaid))#40] +(81) HashAggregate [codegen id : 7] +Input [2]: [sum#44, count#45] +Keys: [] +Functions [1]: [avg(netpaid#31)] +Aggregate Attributes [1]: [avg(netpaid#31)#46] +Results [1]: [(0.05 * avg(netpaid#31)#46) AS (0.05 * avg(netpaid))#47] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index 558a960ab1..336a655002 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -1,16 +1,61 @@ -WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [paid] - Subquery #1 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count] - CometColumnarExchange #9 - CometHashAggregate [netpaid] - CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 +WholeStageCodegen (7) + Filter [paid] + Subquery #1 + WholeStageCodegen (7) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + ColumnarToRow + InputAdapter + CometColumnarExchange #9 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 + CometSort [sr_ticket_number,sr_item_sk] + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 RowToColumnar WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] @@ -27,78 +72,41 @@ WholeStageCodegen (6) CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] CometSort [ss_ticket_number,ss_item_sk] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 + CometColumnarExchange [ss_ticket_number,ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter - BroadcastExchange #11 + BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_color,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - CometHashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - CometHashAggregate [c_last_name,c_first_name,s_store_name,netpaid] - CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometFilter [c_customer_sk,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_country,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometFilter [ca_country,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt index 17f27af6ac..0425915e84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt @@ -1,51 +1,53 @@ == Physical Plan == -* ColumnarToRow (47) -+- CometFilter (46) - +- CometHashAggregate (45) - +- CometColumnarExchange (44) - +- CometHashAggregate (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- RowToColumnar (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometColumnarExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometColumnarExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.item (21) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_address (33) +* Filter (49) ++- * HashAggregate (48) + +- * ColumnarToRow (47) + +- CometColumnarExchange (46) + +- RowToColumnar (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- * ColumnarToRow (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * ColumnarToRow (13) + : : : : +- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.store (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.item (21) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_address (33) (unknown) Scan parquet spark_catalog.default.store_sales @@ -236,103 +238,117 @@ Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_st Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(42) CometHashAggregate +(42) ColumnarToRow [codegen id : 6] +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] + +(43) HashAggregate [codegen id : 6] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] +Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(43) CometHashAggregate -Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#30] +(44) HashAggregate [codegen id : 6] +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] -Functions [1]: [partial_sum(netpaid#30)] +Functions [1]: [partial_sum(netpaid#31)] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] + +(45) RowToColumnar +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(44) CometColumnarExchange -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#31, isEmpty#32] +(46) CometColumnarExchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(45) CometHashAggregate -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#31, isEmpty#32] -Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] -Functions [1]: [sum(netpaid#30)] +(47) ColumnarToRow [codegen id : 7] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(46) CometFilter -Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#33] -Condition : (isnotnull(paid#33) AND (cast(paid#33 as decimal(33,8)) > cast(Subquery scalar-subquery#34, [id=#35] as decimal(33,8)))) +(48) HashAggregate [codegen id : 7] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] +Functions [1]: [sum(netpaid#31)] +Aggregate Attributes [1]: [sum(netpaid#31)#36] +Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(47) ColumnarToRow [codegen id : 6] -Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#33] +(49) Filter [codegen id : 7] +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] +Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#34, [id=#35] -* ColumnarToRow (77) -+- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- CometHashAggregate (74) - +- CometHashAggregate (73) - +- CometColumnarExchange (72) - +- RowToColumnar (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : :- * ColumnarToRow (54) - : : : : +- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (55) - : : +- BroadcastExchange (61) - : : +- * ColumnarToRow (60) - : : +- CometFilter (59) - : : +- CometScan parquet spark_catalog.default.item (58) - : +- ReusedExchange (64) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (81) ++- * ColumnarToRow (80) + +- CometColumnarExchange (79) + +- RowToColumnar (78) + +- * HashAggregate (77) + +- * HashAggregate (76) + +- * ColumnarToRow (75) + +- CometColumnarExchange (74) + +- RowToColumnar (73) + +- * HashAggregate (72) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * Project (65) + : : +- * BroadcastHashJoin Inner BuildRight (64) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * ColumnarToRow (56) + : : : : +- CometProject (55) + : : : : +- CometSortMergeJoin (54) + : : : : :- CometSort (51) + : : : : : +- ReusedExchange (50) + : : : : +- CometSort (53) + : : : : +- ReusedExchange (52) + : : : +- ReusedExchange (57) + : : +- BroadcastExchange (63) + : : +- * ColumnarToRow (62) + : : +- CometFilter (61) + : : +- CometScan parquet spark_catalog.default.item (60) + : +- ReusedExchange (66) + +- ReusedExchange (69) + + +(50) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(49) CometSort +(51) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(50) ReusedExchange [Reuses operator id: 9] +(52) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(51) CometSort +(53) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(52) CometSortMergeJoin +(54) CometSortMergeJoin Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Right output [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(53) CometProject +(55) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(54) ColumnarToRow [codegen id : 5] +(56) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(55) ReusedExchange [Reuses operator id: 18] +(57) ReusedExchange [Reuses operator id: 18] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(56) BroadcastHashJoin [codegen id : 5] +(58) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(57) Project [codegen id : 5] +(59) Project [codegen id : 5] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -343,87 +359,99 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) CometFilter +(61) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : isnotnull(i_item_sk#15) -(60) ColumnarToRow [codegen id : 2] +(62) ColumnarToRow [codegen id : 2] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(61) BroadcastExchange +(63) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(62) BroadcastHashJoin [codegen id : 5] +(64) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(63) Project [codegen id : 5] +(65) Project [codegen id : 5] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(64) ReusedExchange [Reuses operator id: 30] +(66) ReusedExchange [Reuses operator id: 30] Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(65) BroadcastHashJoin [codegen id : 5] +(67) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(66) Project [codegen id : 5] +(68) Project [codegen id : 5] Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(67) ReusedExchange [Reuses operator id: 36] +(69) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#25, ca_zip#26, ca_country#27] -(68) BroadcastHashJoin [codegen id : 5] +(70) BroadcastHashJoin [codegen id : 5] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(69) Project [codegen id : 5] +(71) Project [codegen id : 5] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(70) HashAggregate [codegen id : 5] +(72) HashAggregate [codegen id : 5] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#36] -Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] +Aggregate Attributes [1]: [sum#40] +Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] -(71) RowToColumnar -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] +(73) RowToColumnar +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] -(72) CometColumnarExchange -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] +(74) CometColumnarExchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(73) CometHashAggregate -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#37] +(75) ColumnarToRow [codegen id : 6] +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] + +(76) HashAggregate [codegen id : 6] +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(74) CometHashAggregate -Input [1]: [netpaid#30] +(77) HashAggregate [codegen id : 6] +Input [1]: [netpaid#31] Keys: [] -Functions [1]: [partial_avg(netpaid#30)] +Functions [1]: [partial_avg(netpaid#31)] +Aggregate Attributes [2]: [sum#42, count#43] +Results [2]: [sum#44, count#45] + +(78) RowToColumnar +Input [2]: [sum#44, count#45] -(75) CometColumnarExchange -Input [2]: [sum#38, count#39] +(79) CometColumnarExchange +Input [2]: [sum#44, count#45] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(76) CometHashAggregate -Input [2]: [sum#38, count#39] -Keys: [] -Functions [1]: [avg(netpaid#30)] +(80) ColumnarToRow [codegen id : 7] +Input [2]: [sum#44, count#45] -(77) ColumnarToRow [codegen id : 6] -Input [1]: [(0.05 * avg(netpaid))#40] +(81) HashAggregate [codegen id : 7] +Input [2]: [sum#44, count#45] +Keys: [] +Functions [1]: [avg(netpaid#31)] +Aggregate Attributes [1]: [avg(netpaid#31)#46] +Results [1]: [(0.05 * avg(netpaid#31)#46) AS (0.05 * avg(netpaid))#47] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index 558a960ab1..336a655002 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -1,16 +1,61 @@ -WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [paid] - Subquery #1 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count] - CometColumnarExchange #9 - CometHashAggregate [netpaid] - CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 +WholeStageCodegen (7) + Filter [paid] + Subquery #1 + WholeStageCodegen (7) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + ColumnarToRow + InputAdapter + CometColumnarExchange #9 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 + CometSort [sr_ticket_number,sr_item_sk] + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 RowToColumnar WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] @@ -27,78 +72,41 @@ WholeStageCodegen (6) CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] CometSort [ss_ticket_number,ss_item_sk] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 + CometColumnarExchange [ss_ticket_number,ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter - BroadcastExchange #11 + BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_color,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - CometHashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - CometHashAggregate [c_last_name,c_first_name,s_store_name,netpaid] - CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometFilter [c_customer_sk,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_country,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometFilter [ca_country,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt index 6b203e283f..35c7a35d72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometHashAggregate (40) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- * ColumnarToRow (40) +- CometColumnarExchange (39) +- RowToColumnar (38) +- * HashAggregate (37) @@ -225,17 +225,19 @@ Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31 Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometHashAggregate +(40) ColumnarToRow [codegen id : 9] +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] + +(41) HashAggregate [codegen id : 9] Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#34, sum(UnscaledValue(sr_net_loss#11))#35, sum(UnscaledValue(cs_net_profit#16))#36] +Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#34,17,2) AS store_sales_profit#37, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#35,17,2) AS store_returns_loss#38, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#36,17,2) AS catalog_sales_profit#39] -(41) CometTakeOrderedAndProject -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#34, store_returns_loss#35, catalog_sales_profit#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#26 ASC NULLS FIRST,i_item_desc#27 ASC NULLS FIRST,s_store_id#23 ASC NULLS FIRST,s_store_name#24 ASC NULLS FIRST], output=[i_item_id#26,i_item_desc#27,s_store_id#23,s_store_name#24,store_sales_profit#34,store_returns_loss#35,catalog_sales_profit#36]), 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#34, store_returns_loss#35, catalog_sales_profit#36] - -(42) ColumnarToRow [codegen id : 9] -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#34, store_returns_loss#35, catalog_sales_profit#36] +(42) TakeOrderedAndProject +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] +Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] ===== Subqueries ===== @@ -248,18 +250,18 @@ BroadcastExchange (47) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#37, d_moy#38] +Output [3]: [d_date_sk#19, d_year#40, d_moy#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (44) CometFilter -Input [3]: [d_date_sk#19, d_year#37, d_moy#38] -Condition : ((((isnotnull(d_moy#38) AND isnotnull(d_year#37)) AND (d_moy#38 = 4)) AND (d_year#37 = 2001)) AND isnotnull(d_date_sk#19)) +Input [3]: [d_date_sk#19, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 4)) AND (d_year#40 = 2001)) AND isnotnull(d_date_sk#19)) (45) CometProject -Input [3]: [d_date_sk#19, d_year#37, d_moy#38] +Input [3]: [d_date_sk#19, d_year#40, d_moy#41] Arguments: [d_date_sk#19], [d_date_sk#19] (46) ColumnarToRow [codegen id : 1] @@ -278,18 +280,18 @@ BroadcastExchange (52) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#39, d_moy#40] +Output [3]: [d_date_sk#20, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (49) CometFilter -Input [3]: [d_date_sk#20, d_year#39, d_moy#40] -Condition : (((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 >= 4)) AND (d_moy#40 <= 10)) AND (d_year#39 = 2001)) AND isnotnull(d_date_sk#20)) +Input [3]: [d_date_sk#20, d_year#42, d_moy#43] +Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 4)) AND (d_moy#43 <= 10)) AND (d_year#42 = 2001)) AND isnotnull(d_date_sk#20)) (50) CometProject -Input [3]: [d_date_sk#20, d_year#39, d_moy#40] +Input [3]: [d_date_sk#20, d_year#42, d_moy#43] Arguments: [d_date_sk#20], [d_date_sk#20] (51) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index 9d8f0b4415..0418c4a7e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] +TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] + WholeStageCodegen (9) + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 RowToColumnar WholeStageCodegen (8) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index cdb47fded7..192aacb4cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (32) -+- CometTakeOrderedAndProject (31) - +- CometHashAggregate (30) +TakeOrderedAndProject (32) ++- * HashAggregate (31) + +- * ColumnarToRow (30) +- CometColumnarExchange (29) +- RowToColumnar (28) +- * HashAggregate (27) @@ -167,17 +167,19 @@ Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometHashAggregate +(30) ColumnarToRow [codegen id : 6] +Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] + +(31) HashAggregate [codegen id : 6] Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] Keys [1]: [i_item_id#16] Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [4]: [avg(cs_quantity#4)#36, avg(UnscaledValue(cs_list_price#5))#37, avg(UnscaledValue(cs_coupon_amt#7))#38, avg(UnscaledValue(cs_sales_price#6))#39] +Results [5]: [i_item_id#16, avg(cs_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(cs_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(cs_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(cs_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] -(31) CometTakeOrderedAndProject -Input [5]: [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#16 ASC NULLS FIRST], output=[i_item_id#16,agg1#36,agg2#37,agg3#38,agg4#39]), 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] - -(32) ColumnarToRow [codegen id : 6] -Input [5]: [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] +(32) TakeOrderedAndProject +Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] +Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] ===== Subqueries ===== @@ -190,18 +192,18 @@ BroadcastExchange (37) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#40] +Output [2]: [d_date_sk#14, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (34) CometFilter -Input [2]: [d_date_sk#14, d_year#40] -Condition : ((isnotnull(d_year#40) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) (35) CometProject -Input [2]: [d_date_sk#14, d_year#40] +Input [2]: [d_date_sk#14, d_year#44] Arguments: [d_date_sk#14], [d_date_sk#14] (36) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index 1d9e320882..67e6093dfc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] +TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] + WholeStageCodegen (6) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #1 RowToColumnar WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index 7a4a7aaf9b..0e3e6c9917 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (32) -+- CometTakeOrderedAndProject (31) - +- CometHashAggregate (30) +TakeOrderedAndProject (32) ++- * HashAggregate (31) + +- * ColumnarToRow (30) +- CometColumnarExchange (29) +- RowToColumnar (28) +- * HashAggregate (27) @@ -167,17 +167,19 @@ Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, s Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] Arguments: hashpartitioning(i_item_id#19, s_state#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometHashAggregate +(30) ColumnarToRow [codegen id : 6] +Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] + +(31) HashAggregate [codegen id : 6] Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] Keys [3]: [i_item_id#19, s_state#20, spark_grouping_id#21] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [4]: [avg(ss_quantity#4)#38, avg(UnscaledValue(ss_list_price#5))#39, avg(UnscaledValue(ss_coupon_amt#7))#40, avg(UnscaledValue(ss_sales_price#6))#41] +Results [7]: [i_item_id#19, s_state#20, cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) AS g_state#42, avg(ss_quantity#4)#38 AS agg1#43, cast((avg(UnscaledValue(ss_list_price#5))#39 / 100.0) as decimal(11,6)) AS agg2#44, cast((avg(UnscaledValue(ss_coupon_amt#7))#40 / 100.0) as decimal(11,6)) AS agg3#45, cast((avg(UnscaledValue(ss_sales_price#6))#41 / 100.0) as decimal(11,6)) AS agg4#46] -(31) CometTakeOrderedAndProject -Input [7]: [i_item_id#19, s_state#20, g_state#38, agg1#39, agg2#40, agg3#41, agg4#42] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#19 ASC NULLS FIRST,s_state#20 ASC NULLS FIRST], output=[i_item_id#19,s_state#20,g_state#38,agg1#39,agg2#40,agg3#41,agg4#42]), 100, [i_item_id#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST], [i_item_id#19, s_state#20, g_state#38, agg1#39, agg2#40, agg3#41, agg4#42] - -(32) ColumnarToRow [codegen id : 6] -Input [7]: [i_item_id#19, s_state#20, g_state#38, agg1#39, agg2#40, agg3#41, agg4#42] +(32) TakeOrderedAndProject +Input [7]: [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] +Arguments: 100, [i_item_id#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST], [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] ===== Subqueries ===== @@ -190,18 +192,18 @@ BroadcastExchange (37) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#43] +Output [2]: [d_date_sk#14, d_year#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (34) CometFilter -Input [2]: [d_date_sk#14, d_year#43] -Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2002)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_year#47] +Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#14)) (35) CometProject -Input [2]: [d_date_sk#14, d_year#43] +Input [2]: [d_date_sk#14, d_year#47] Arguments: [d_date_sk#14], [d_date_sk#14] (36) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index 191ff0322c..f57e4c443a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] +TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + WholeStageCodegen (6) + HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id,s_state,spark_grouping_id] #1 RowToColumnar WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt index 69e36471f3..3a3000118a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometHashAggregate (40) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- * ColumnarToRow (40) +- CometColumnarExchange (39) +- RowToColumnar (38) +- * HashAggregate (37) @@ -225,17 +225,19 @@ Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31 Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometHashAggregate +(40) ColumnarToRow [codegen id : 9] +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] + +(41) HashAggregate [codegen id : 9] Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] +Aggregate Attributes [3]: [sum(ss_quantity#5)#34, sum(sr_return_quantity#11)#35, sum(cs_quantity#16)#36] +Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum(ss_quantity#5)#34 AS store_sales_quantity#37, sum(sr_return_quantity#11)#35 AS store_returns_quantity#38, sum(cs_quantity#16)#36 AS catalog_sales_quantity#39] -(41) CometTakeOrderedAndProject -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#34, store_returns_quantity#35, catalog_sales_quantity#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#26 ASC NULLS FIRST,i_item_desc#27 ASC NULLS FIRST,s_store_id#23 ASC NULLS FIRST,s_store_name#24 ASC NULLS FIRST], output=[i_item_id#26,i_item_desc#27,s_store_id#23,s_store_name#24,store_sales_quantity#34,store_returns_quantity#35,catalog_sales_quantity#36]), 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#34, store_returns_quantity#35, catalog_sales_quantity#36] - -(42) ColumnarToRow [codegen id : 9] -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#34, store_returns_quantity#35, catalog_sales_quantity#36] +(42) TakeOrderedAndProject +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] +Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] ===== Subqueries ===== @@ -248,18 +250,18 @@ BroadcastExchange (47) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#37, d_moy#38] +Output [3]: [d_date_sk#19, d_year#40, d_moy#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct (44) CometFilter -Input [3]: [d_date_sk#19, d_year#37, d_moy#38] -Condition : ((((isnotnull(d_moy#38) AND isnotnull(d_year#37)) AND (d_moy#38 = 9)) AND (d_year#37 = 1999)) AND isnotnull(d_date_sk#19)) +Input [3]: [d_date_sk#19, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 9)) AND (d_year#40 = 1999)) AND isnotnull(d_date_sk#19)) (45) CometProject -Input [3]: [d_date_sk#19, d_year#37, d_moy#38] +Input [3]: [d_date_sk#19, d_year#40, d_moy#41] Arguments: [d_date_sk#19], [d_date_sk#19] (46) ColumnarToRow [codegen id : 1] @@ -278,18 +280,18 @@ BroadcastExchange (52) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#39, d_moy#40] +Output [3]: [d_date_sk#20, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct (49) CometFilter -Input [3]: [d_date_sk#20, d_year#39, d_moy#40] -Condition : (((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 >= 9)) AND (d_moy#40 <= 12)) AND (d_year#39 = 1999)) AND isnotnull(d_date_sk#20)) +Input [3]: [d_date_sk#20, d_year#42, d_moy#43] +Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 9)) AND (d_moy#43 <= 12)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#20)) (50) CometProject -Input [3]: [d_date_sk#20, d_year#39, d_moy#40] +Input [3]: [d_date_sk#20, d_year#42, d_moy#43] Arguments: [d_date_sk#20], [d_date_sk#20] (51) ColumnarToRow [codegen id : 1] @@ -308,18 +310,18 @@ BroadcastExchange (57) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_year#41] +Output [2]: [d_date_sk#21, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct (54) CometFilter -Input [2]: [d_date_sk#21, d_year#41] -Condition : (d_year#41 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) +Input [2]: [d_date_sk#21, d_year#44] +Condition : (d_year#44 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) (55) CometProject -Input [2]: [d_date_sk#21, d_year#41] +Input [2]: [d_date_sk#21, d_year#44] Arguments: [d_date_sk#21], [d_date_sk#21] (56) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt index 753b11f806..9de3a02a39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] +TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] + WholeStageCodegen (9) + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 RowToColumnar WholeStageCodegen (8) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt index 2baeadfd8b..dab78aab5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometHashAggregate (21) +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- * ColumnarToRow (21) +- CometColumnarExchange (20) +- RowToColumnar (19) +- * HashAggregate (18) @@ -117,15 +117,17 @@ Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometHashAggregate +(21) ColumnarToRow [codegen id : 4] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] + +(22) HashAggregate [codegen id : 4] Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] +Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum_agg#16] -(22) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), 100, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] - -(23) ColumnarToRow [codegen id : 4] -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] +(23) TakeOrderedAndProject +Input [4]: [d_year#2, brand_id#14, brand#15, sum_agg#16] +Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, sum_agg#16] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt index a627b6b25e..191bf4c810 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,sum_agg,brand_id,brand] - CometHashAggregate [d_year,i_brand,i_brand_id,sum] +TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] + WholeStageCodegen (4) + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [d_year,i_brand,i_brand_id] #1 RowToColumnar WholeStageCodegen (3) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index 13d06f87c1..2b3ab365c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (53) -+- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometHashAggregate (16) +TakeOrderedAndProject (55) ++- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (47) + : +- * BroadcastHashJoin Inner BuildRight (46) + : :- * Project (41) + : : +- * BroadcastHashJoin Inner BuildRight (40) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * ColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- RowToColumnar (14) : : : +- * HashAggregate (13) @@ -24,34 +24,36 @@ TakeOrderedAndProject (53) : : : +- * ColumnarToRow (9) : : : +- CometFilter (8) : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (37) - : : +- * ColumnarToRow (36) - : : +- CometFilter (35) - : : +- CometHashAggregate (34) - : : +- CometColumnarExchange (33) - : : +- CometHashAggregate (32) - : : +- CometHashAggregate (31) - : : +- CometColumnarExchange (30) - : : +- RowToColumnar (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.web_returns (19) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (43) - : +- * ColumnarToRow (42) - : +- CometFilter (41) - : +- CometScan parquet spark_catalog.default.customer (40) - +- BroadcastExchange (50) - +- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.customer_address (46) + : : +- BroadcastExchange (39) + : : +- * Filter (38) + : : +- * HashAggregate (37) + : : +- * ColumnarToRow (36) + : : +- CometColumnarExchange (35) + : : +- RowToColumnar (34) + : : +- * HashAggregate (33) + : : +- * HashAggregate (32) + : : +- * ColumnarToRow (31) + : : +- CometColumnarExchange (30) + : : +- RowToColumnar (29) + : : +- * HashAggregate (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (24) + : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : :- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.web_returns (19) + : : : +- ReusedExchange (22) + : : +- ReusedExchange (25) + : +- BroadcastExchange (45) + : +- * ColumnarToRow (44) + : +- CometFilter (43) + : +- CometScan parquet spark_catalog.default.customer (42) + +- BroadcastExchange (52) + +- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.customer_address (48) (unknown) Scan parquet spark_catalog.default.web_returns @@ -69,7 +71,7 @@ Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_custom (3) ColumnarToRow [codegen id : 3] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -(4) ReusedExchange [Reuses operator id: 58] +(4) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -124,23 +126,25 @@ Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 11] +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] + +(17) HashAggregate [codegen id : 11] Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] Keys [2]: [wr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#11] +Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] -(17) CometFilter -Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] -Condition : isnotnull(ctr_total_return#13) - -(18) ColumnarToRow [codegen id : 10] -Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +(18) Filter [codegen id : 11] +Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Condition : isnotnull(ctr_total_return#14) (unknown) Scan parquet spark_catalog.default.web_returns Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#14)] +PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#15)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct @@ -151,7 +155,7 @@ Condition : isnotnull(wr_returning_addr_sk#2) (21) ColumnarToRow [codegen id : 6] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -(22) ReusedExchange [Reuses operator id: 58] +(22) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#6] (23) BroadcastHashJoin [codegen id : 6] @@ -181,149 +185,161 @@ Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] Keys [2]: [wr_returning_customer_sk#1, ca_state#8] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#15] -Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] (29) RowToColumnar -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#16] +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] (30) CometColumnarExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#16] +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(31) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#16] +(31) ColumnarToRow [codegen id : 7] +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] + +(32) HashAggregate [codegen id : 7] +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] Keys [2]: [wr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] - -(32) CometHashAggregate -Input [2]: [ctr_state#12, ctr_total_return#13] -Keys [1]: [ctr_state#12] -Functions [1]: [partial_avg(ctr_total_return#13)] - -(33) CometColumnarExchange -Input [3]: [ctr_state#12, sum#17, count#18] -Arguments: hashpartitioning(ctr_state#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [3]: [ctr_state#12, sum#17, count#18] -Keys [1]: [ctr_state#12] -Functions [1]: [avg(ctr_total_return#13)] - -(35) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#19) - -(36) ColumnarToRow [codegen id : 7] -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#11] +Results [2]: [ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] + +(33) HashAggregate [codegen id : 7] +Input [2]: [ctr_state#13, ctr_total_return#14] +Keys [1]: [ctr_state#13] +Functions [1]: [partial_avg(ctr_total_return#14)] +Aggregate Attributes [2]: [sum#18, count#19] +Results [3]: [ctr_state#13, sum#20, count#21] + +(34) RowToColumnar +Input [3]: [ctr_state#13, sum#20, count#21] + +(35) CometColumnarExchange +Input [3]: [ctr_state#13, sum#20, count#21] +Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(36) ColumnarToRow [codegen id : 8] +Input [3]: [ctr_state#13, sum#20, count#21] + +(37) HashAggregate [codegen id : 8] +Input [3]: [ctr_state#13, sum#20, count#21] +Keys [1]: [ctr_state#13] +Functions [1]: [avg(ctr_total_return#14)] +Aggregate Attributes [1]: [avg(ctr_total_return#14)#22] +Results [2]: [(avg(ctr_total_return#14)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#13 AS ctr_state#13#24] + +(38) Filter [codegen id : 8] +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) + +(39) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ctr_state#12] -Right keys [1]: [ctr_state#12#20] +(40) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ctr_state#13] +Right keys [1]: [ctr_state#13#24] Join type: Inner -Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#19) +Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) -(39) Project [codegen id : 10] -Output [2]: [ctr_customer_sk#11, ctr_total_return#13] -Input [5]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] +(41) Project [codegen id : 11] +Output [2]: [ctr_customer_sk#12, ctr_total_return#14] +Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] (unknown) Scan parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] +Output [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(41) CometFilter -Input [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#23)) +(43) CometFilter +Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] +Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) -(42) ColumnarToRow [codegen id : 8] -Input [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] +(44) ColumnarToRow [codegen id : 9] +Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] -(43) BroadcastExchange -Input [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] +(45) BroadcastExchange +Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ctr_customer_sk#11] -Right keys [1]: [c_customer_sk#21] +(46) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ctr_customer_sk#12] +Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(45) Project [codegen id : 10] -Output [14]: [ctr_total_return#13, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] -Input [16]: [ctr_customer_sk#11, ctr_total_return#13, c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] +(47) Project [codegen id : 11] +Output [14]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] +Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] (unknown) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#35, ca_state#36] +Output [2]: [ca_address_sk#39, ca_state#40] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(47) CometFilter -Input [2]: [ca_address_sk#35, ca_state#36] -Condition : ((isnotnull(ca_state#36) AND (ca_state#36 = GA)) AND isnotnull(ca_address_sk#35)) +(49) CometFilter +Input [2]: [ca_address_sk#39, ca_state#40] +Condition : ((isnotnull(ca_state#40) AND (ca_state#40 = GA)) AND isnotnull(ca_address_sk#39)) -(48) CometProject -Input [2]: [ca_address_sk#35, ca_state#36] -Arguments: [ca_address_sk#35], [ca_address_sk#35] +(50) CometProject +Input [2]: [ca_address_sk#39, ca_state#40] +Arguments: [ca_address_sk#39], [ca_address_sk#39] -(49) ColumnarToRow [codegen id : 9] -Input [1]: [ca_address_sk#35] +(51) ColumnarToRow [codegen id : 10] +Input [1]: [ca_address_sk#39] -(50) BroadcastExchange -Input [1]: [ca_address_sk#35] +(52) BroadcastExchange +Input [1]: [ca_address_sk#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_current_addr_sk#23] -Right keys [1]: [ca_address_sk#35] +(53) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#27] +Right keys [1]: [ca_address_sk#39] Join type: Inner Join condition: None -(52) Project [codegen id : 10] -Output [13]: [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34, ctr_total_return#13] -Input [15]: [ctr_total_return#13, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34, ca_address_sk#35] +(54) Project [codegen id : 11] +Output [13]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] +Input [15]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ca_address_sk#39] -(53) TakeOrderedAndProject -Input [13]: [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34, ctr_total_return#13] -Arguments: 100, [c_customer_id#22 ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_first_name#25 ASC NULLS FIRST, c_last_name#26 ASC NULLS FIRST, c_preferred_cust_flag#27 ASC NULLS FIRST, c_birth_day#28 ASC NULLS FIRST, c_birth_month#29 ASC NULLS FIRST, c_birth_year#30 ASC NULLS FIRST, c_birth_country#31 ASC NULLS FIRST, c_login#32 ASC NULLS FIRST, c_email_address#33 ASC NULLS FIRST, c_last_review_date#34 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34, ctr_total_return#13] +(55) TakeOrderedAndProject +Input [13]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] +Arguments: 100, [c_customer_id#26 ASC NULLS FIRST, c_salutation#28 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, c_last_name#30 ASC NULLS FIRST, c_preferred_cust_flag#31 ASC NULLS FIRST, c_birth_day#32 ASC NULLS FIRST, c_birth_month#33 ASC NULLS FIRST, c_birth_year#34 ASC NULLS FIRST, c_birth_country#35 ASC NULLS FIRST, c_login#36 ASC NULLS FIRST, c_email_address#37 ASC NULLS FIRST, c_last_review_date#38 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (58) -+- * ColumnarToRow (57) - +- CometProject (56) - +- CometFilter (55) - +- CometScan parquet spark_catalog.default.date_dim (54) +BroadcastExchange (60) ++- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#37] +Output [2]: [d_date_sk#6, d_year#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(55) CometFilter -Input [2]: [d_date_sk#6, d_year#37] -Condition : ((isnotnull(d_year#37) AND (d_year#37 = 2002)) AND isnotnull(d_date_sk#6)) +(57) CometFilter +Input [2]: [d_date_sk#6, d_year#41] +Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#6)) -(56) CometProject -Input [2]: [d_date_sk#6, d_year#37] +(58) CometProject +Input [2]: [d_date_sk#6, d_year#41] Arguments: [d_date_sk#6], [d_date_sk#6] -(57) ColumnarToRow [codegen id : 1] +(59) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(58) BroadcastExchange +(60) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index a883ad9d96..d6458b01ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -1,15 +1,15 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (10) + WholeStageCodegen (11) Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk,ctr_total_return] BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - ColumnarToRow - InputAdapter - CometFilter [ctr_total_return] - CometHashAggregate [wr_returning_customer_sk,ca_state,sum] + Filter [ctr_total_return] + HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 RowToColumnar WholeStageCodegen (3) @@ -41,41 +41,45 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [(avg(ctr_total_return) * 1.2)] - CometHashAggregate [ctr_state,sum,count] + WholeStageCodegen (8) + Filter [(avg(ctr_total_return) * 1.2)] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ctr_state] #5 - CometHashAggregate [ctr_state,ctr_total_return] - CometHashAggregate [wr_returning_customer_sk,ca_state,sum] - CometColumnarExchange [wr_returning_customer_sk,ca_state] #6 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [wr_returning_addr_sk] - CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [wr_returning_customer_sk,ca_state] #6 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_returning_addr_sk] + CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #8 - WholeStageCodegen (9) + WholeStageCodegen (10) ColumnarToRow InputAdapter CometProject [ca_address_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index ccfa5eb025..da7d8fd881 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt @@ -11,8 +11,8 @@ : : :- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) : : : :- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * ColumnarToRow (17) - : : : : : +- CometHashAggregate (16) + : : : : :- * HashAggregate (17) + : : : : : +- * ColumnarToRow (16) : : : : : +- CometColumnarExchange (15) : : : : : +- RowToColumnar (14) : : : : : +- * HashAggregate (13) @@ -29,8 +29,8 @@ : : : : : +- CometFilter (8) : : : : : +- CometScan parquet spark_catalog.default.customer_address (7) : : : : +- BroadcastExchange (32) - : : : : +- * ColumnarToRow (31) - : : : : +- CometHashAggregate (30) + : : : : +- * HashAggregate (31) + : : : : +- * ColumnarToRow (30) : : : : +- CometColumnarExchange (29) : : : : +- RowToColumnar (28) : : : : +- * HashAggregate (27) @@ -44,8 +44,8 @@ : : : : : +- ReusedExchange (21) : : : : +- ReusedExchange (24) : : : +- BroadcastExchange (48) - : : : +- * ColumnarToRow (47) - : : : +- CometHashAggregate (46) + : : : +- * HashAggregate (47) + : : : +- * ColumnarToRow (46) : : : +- CometColumnarExchange (45) : : : +- RowToColumnar (44) : : : +- * HashAggregate (43) @@ -59,8 +59,8 @@ : : : : +- ReusedExchange (37) : : : +- ReusedExchange (40) : : +- BroadcastExchange (65) - : : +- * ColumnarToRow (64) - : : +- CometHashAggregate (63) + : : +- * HashAggregate (64) + : : +- * ColumnarToRow (63) : : +- CometColumnarExchange (62) : : +- RowToColumnar (61) : : +- * HashAggregate (60) @@ -74,8 +74,8 @@ : : : +- ReusedExchange (54) : : +- ReusedExchange (57) : +- BroadcastExchange (81) - : +- * ColumnarToRow (80) - : +- CometHashAggregate (79) + : +- * HashAggregate (80) + : +- * ColumnarToRow (79) : +- CometColumnarExchange (78) : +- RowToColumnar (77) : +- * HashAggregate (76) @@ -89,8 +89,8 @@ : : +- ReusedExchange (70) : +- ReusedExchange (73) +- BroadcastExchange (98) - +- * ColumnarToRow (97) - +- CometHashAggregate (96) + +- * HashAggregate (97) + +- * ColumnarToRow (96) +- CometColumnarExchange (95) +- RowToColumnar (94) +- * HashAggregate (93) @@ -175,404 +175,416 @@ Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 24] +Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] + +(17) HashAggregate [codegen id : 24] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(17) ColumnarToRow [codegen id : 24] -Input [3]: [ca_county#9, d_year#6, store_sales#12] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] +Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS store_sales#13] (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] +Output [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#17)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (19) CometFilter -Input [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_addr_sk#13) +Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] +Condition : isnotnull(ss_addr_sk#14) (20) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] +Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] (21) ReusedExchange [Reuses operator id: 112] -Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] (22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ss_sold_date_sk#16] +Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None (23) Project [codegen id : 6] -Output [4]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19] -Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_year#18, d_qoy#19] +Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] +Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] (24) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#20, ca_county#21] +Output [2]: [ca_address_sk#21, ca_county#22] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#13] -Right keys [1]: [ca_address_sk#20] +Left keys [1]: [ss_addr_sk#14] +Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None (26) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] -Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_address_sk#20, ca_county#21] +Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] +Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] (27) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] -Keys [3]: [ca_county#21, d_qoy#19, d_year#18] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#14))] -Aggregate Attributes [1]: [sum#22] -Results [4]: [ca_county#21, d_qoy#19, d_year#18, sum#23] +Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] +Keys [3]: [ca_county#22, d_qoy#20, d_year#19] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] +Aggregate Attributes [1]: [sum#23] +Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] (28) RowToColumnar -Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#23] +Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] (29) CometColumnarExchange -Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#23] -Arguments: hashpartitioning(ca_county#21, d_qoy#19, d_year#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] +Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometHashAggregate -Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#23] -Keys [3]: [ca_county#21, d_qoy#19, d_year#18] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#14))] +(30) ColumnarToRow [codegen id : 7] +Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -(31) ColumnarToRow [codegen id : 7] -Input [2]: [ca_county#21, store_sales#24] +(31) HashAggregate [codegen id : 7] +Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] +Keys [3]: [ca_county#22, d_qoy#20, d_year#19] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#15))#12] +Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#12,17,2) AS store_sales#25] (32) BroadcastExchange -Input [2]: [ca_county#21, store_sales#24] +Input [2]: [ca_county#22, store_sales#25] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] (33) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#21] +Right keys [1]: [ca_county#22] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27] +Output [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_sold_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (35) CometFilter -Input [3]: [ss_addr_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27] -Condition : isnotnull(ss_addr_sk#25) +Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Condition : isnotnull(ss_addr_sk#26) (36) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27] +Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] (37) ReusedExchange [Reuses operator id: 116] -Output [3]: [d_date_sk#29, d_year#30, d_qoy#31] +Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#27] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [ss_sold_date_sk#28] +Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None (39) Project [codegen id : 10] -Output [4]: [ss_addr_sk#25, ss_ext_sales_price#26, d_year#30, d_qoy#31] -Input [6]: [ss_addr_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27, d_date_sk#29, d_year#30, d_qoy#31] +Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] +Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] (40) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#32, ca_county#33] +Output [2]: [ca_address_sk#33, ca_county#34] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#25] -Right keys [1]: [ca_address_sk#32] +Left keys [1]: [ss_addr_sk#26] +Right keys [1]: [ca_address_sk#33] Join type: Inner Join condition: None (42) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#26, d_year#30, d_qoy#31, ca_county#33] -Input [6]: [ss_addr_sk#25, ss_ext_sales_price#26, d_year#30, d_qoy#31, ca_address_sk#32, ca_county#33] +Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] +Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] (43) HashAggregate [codegen id : 10] -Input [4]: [ss_ext_sales_price#26, d_year#30, d_qoy#31, ca_county#33] -Keys [3]: [ca_county#33, d_qoy#31, d_year#30] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [4]: [ca_county#33, d_qoy#31, d_year#30, sum#35] +Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] +Keys [3]: [ca_county#34, d_qoy#32, d_year#31] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] +Aggregate Attributes [1]: [sum#35] +Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] (44) RowToColumnar -Input [4]: [ca_county#33, d_qoy#31, d_year#30, sum#35] +Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] (45) CometColumnarExchange -Input [4]: [ca_county#33, d_qoy#31, d_year#30, sum#35] -Arguments: hashpartitioning(ca_county#33, d_qoy#31, d_year#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] +Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(46) CometHashAggregate -Input [4]: [ca_county#33, d_qoy#31, d_year#30, sum#35] -Keys [3]: [ca_county#33, d_qoy#31, d_year#30] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#26))] +(46) ColumnarToRow [codegen id : 11] +Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -(47) ColumnarToRow [codegen id : 11] -Input [2]: [ca_county#33, store_sales#36] +(47) HashAggregate [codegen id : 11] +Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] +Keys [3]: [ca_county#34, d_qoy#32, d_year#31] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#12] +Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#12,17,2) AS store_sales#37] (48) BroadcastExchange -Input [2]: [ca_county#33, store_sales#36] +Input [2]: [ca_county#34, store_sales#37] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] (49) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#21] -Right keys [1]: [ca_county#33] +Left keys [1]: [ca_county#22] +Right keys [1]: [ca_county#34] Join type: Inner Join condition: None (50) Project [codegen id : 24] -Output [5]: [ca_county#9, d_year#6, store_sales#12, store_sales#24, store_sales#36] -Input [7]: [ca_county#9, d_year#6, store_sales#12, ca_county#21, store_sales#24, ca_county#33, store_sales#36] +Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] +Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, ws_sold_date_sk#39] +Output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#39), dynamicpruningexpression(ws_sold_date_sk#39 IN dynamicpruning#40)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#41)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (52) CometFilter -Input [3]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, ws_sold_date_sk#39] -Condition : isnotnull(ws_bill_addr_sk#37) +Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : isnotnull(ws_bill_addr_sk#38) (53) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, ws_sold_date_sk#39] +Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (54) ReusedExchange [Reuses operator id: 108] -Output [3]: [d_date_sk#41, d_year#42, d_qoy#43] +Output [3]: [d_date_sk#42, d_year#43, d_qoy#44] (55) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#39] -Right keys [1]: [d_date_sk#41] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#42] Join type: Inner Join condition: None (56) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, d_year#42, d_qoy#43] -Input [6]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, ws_sold_date_sk#39, d_date_sk#41, d_year#42, d_qoy#43] +Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44] +Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#42, d_year#43, d_qoy#44] (57) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#44, ca_county#45] +Output [2]: [ca_address_sk#45, ca_county#46] (58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#37] -Right keys [1]: [ca_address_sk#44] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#45] Join type: Inner Join condition: None (59) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#38, d_year#42, d_qoy#43, ca_county#45] -Input [6]: [ws_bill_addr_sk#37, ws_ext_sales_price#38, d_year#42, d_qoy#43, ca_address_sk#44, ca_county#45] +Output [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] +Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_address_sk#45, ca_county#46] (60) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#38, d_year#42, d_qoy#43, ca_county#45] -Keys [3]: [ca_county#45, d_qoy#43, d_year#42] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#38))] -Aggregate Attributes [1]: [sum#46] -Results [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] +Input [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] +Keys [3]: [ca_county#46, d_qoy#44, d_year#43] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#47] +Results [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] (61) RowToColumnar -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] +Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] (62) CometColumnarExchange -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] -Arguments: hashpartitioning(ca_county#45, d_qoy#43, d_year#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] +Arguments: hashpartitioning(ca_county#46, d_qoy#44, d_year#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(63) CometHashAggregate -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] -Keys [3]: [ca_county#45, d_qoy#43, d_year#42] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#38))] +(63) ColumnarToRow [codegen id : 15] +Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] -(64) ColumnarToRow [codegen id : 15] -Input [2]: [ca_county#45, web_sales#48] +(64) HashAggregate [codegen id : 15] +Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] +Keys [3]: [ca_county#46, d_qoy#44, d_year#43] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#49] +Results [2]: [ca_county#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#49,17,2) AS web_sales#50] (65) BroadcastExchange -Input [2]: [ca_county#45, web_sales#48] +Input [2]: [ca_county#46, web_sales#50] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] (66) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#45] +Right keys [1]: [ca_county#46] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Output [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (68) CometFilter -Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_bill_addr_sk#49) +Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_bill_addr_sk#51) (69) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] (70) ReusedExchange [Reuses operator id: 112] -Output [3]: [d_date_sk#53, d_year#54, d_qoy#55] +Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] (71) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#53] +Left keys [1]: [ws_sold_date_sk#53] +Right keys [1]: [d_date_sk#55] Join type: Inner Join condition: None (72) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#54, d_qoy#55] -Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51, d_date_sk#53, d_year#54, d_qoy#55] +Output [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57] +Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56, d_qoy#57] (73) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#56, ca_county#57] +Output [2]: [ca_address_sk#58, ca_county#59] (74) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#49] -Right keys [1]: [ca_address_sk#56] +Left keys [1]: [ws_bill_addr_sk#51] +Right keys [1]: [ca_address_sk#58] Join type: Inner Join condition: None (75) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#50, d_year#54, d_qoy#55, ca_county#57] -Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#54, d_qoy#55, ca_address_sk#56, ca_county#57] +Output [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] +Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_address_sk#58, ca_county#59] (76) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#50, d_year#54, d_qoy#55, ca_county#57] -Keys [3]: [ca_county#57, d_qoy#55, d_year#54] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] -Aggregate Attributes [1]: [sum#58] -Results [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] +Input [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] +Keys [3]: [ca_county#59, d_qoy#57, d_year#56] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#52))] +Aggregate Attributes [1]: [sum#60] +Results [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] (77) RowToColumnar -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] +Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] (78) CometColumnarExchange -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] -Arguments: hashpartitioning(ca_county#57, d_qoy#55, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] +Arguments: hashpartitioning(ca_county#59, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(79) CometHashAggregate -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] -Keys [3]: [ca_county#57, d_qoy#55, d_year#54] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#50))] +(79) ColumnarToRow [codegen id : 19] +Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] -(80) ColumnarToRow [codegen id : 19] -Input [2]: [ca_county#57, web_sales#60] +(80) HashAggregate [codegen id : 19] +Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] +Keys [3]: [ca_county#59, d_qoy#57, d_year#56] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#52))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#52))#49] +Results [2]: [ca_county#59, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#52))#49,17,2) AS web_sales#62] (81) BroadcastExchange -Input [2]: [ca_county#57, web_sales#60] +Input [2]: [ca_county#59, web_sales#62] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] (82) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#45] -Right keys [1]: [ca_county#57] +Left keys [1]: [ca_county#46] +Right keys [1]: [ca_county#59] Join type: Inner -Join condition: (CASE WHEN (web_sales#48 > 0.00) THEN (web_sales#60 / web_sales#48) END > CASE WHEN (store_sales#12 > 0.00) THEN (store_sales#24 / store_sales#12) END) +Join condition: (CASE WHEN (web_sales#50 > 0.00) THEN (web_sales#62 / web_sales#50) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) (83) Project [codegen id : 24] -Output [8]: [ca_county#9, d_year#6, store_sales#12, store_sales#24, store_sales#36, ca_county#45, web_sales#48, web_sales#60] -Input [9]: [ca_county#9, d_year#6, store_sales#12, store_sales#24, store_sales#36, ca_county#45, web_sales#48, ca_county#57, web_sales#60] +Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62] +Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, ca_county#59, web_sales#62] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] +Output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#66)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (85) CometFilter -Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_addr_sk#61) +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_addr_sk#63) (86) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] (87) ReusedExchange [Reuses operator id: 116] -Output [3]: [d_date_sk#65, d_year#66, d_qoy#67] +Output [3]: [d_date_sk#67, d_year#68, d_qoy#69] (88) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#65] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#67] Join type: Inner Join condition: None (89) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#66, d_qoy#67] -Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66, d_qoy#67] +Output [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69] +Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68, d_qoy#69] (90) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#68, ca_county#69] +Output [2]: [ca_address_sk#70, ca_county#71] (91) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#61] -Right keys [1]: [ca_address_sk#68] +Left keys [1]: [ws_bill_addr_sk#63] +Right keys [1]: [ca_address_sk#70] Join type: Inner Join condition: None (92) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#62, d_year#66, d_qoy#67, ca_county#69] -Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#66, d_qoy#67, ca_address_sk#68, ca_county#69] +Output [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] +Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_address_sk#70, ca_county#71] (93) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#62, d_year#66, d_qoy#67, ca_county#69] -Keys [3]: [ca_county#69, d_qoy#67, d_year#66] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#62))] -Aggregate Attributes [1]: [sum#70] -Results [4]: [ca_county#69, d_qoy#67, d_year#66, sum#71] +Input [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] +Keys [3]: [ca_county#71, d_qoy#69, d_year#68] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] +Aggregate Attributes [1]: [sum#72] +Results [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] (94) RowToColumnar -Input [4]: [ca_county#69, d_qoy#67, d_year#66, sum#71] +Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] (95) CometColumnarExchange -Input [4]: [ca_county#69, d_qoy#67, d_year#66, sum#71] -Arguments: hashpartitioning(ca_county#69, d_qoy#67, d_year#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] +Arguments: hashpartitioning(ca_county#71, d_qoy#69, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(96) CometHashAggregate -Input [4]: [ca_county#69, d_qoy#67, d_year#66, sum#71] -Keys [3]: [ca_county#69, d_qoy#67, d_year#66] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#62))] +(96) ColumnarToRow [codegen id : 23] +Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] -(97) ColumnarToRow [codegen id : 23] -Input [2]: [ca_county#69, web_sales#72] +(97) HashAggregate [codegen id : 23] +Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] +Keys [3]: [ca_county#71, d_qoy#69, d_year#68] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#64))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#64))#49] +Results [2]: [ca_county#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#49,17,2) AS web_sales#74] (98) BroadcastExchange -Input [2]: [ca_county#69, web_sales#72] +Input [2]: [ca_county#71, web_sales#74] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] (99) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#45] -Right keys [1]: [ca_county#69] +Left keys [1]: [ca_county#46] +Right keys [1]: [ca_county#71] Join type: Inner -Join condition: (CASE WHEN (web_sales#60 > 0.00) THEN (web_sales#72 / web_sales#60) END > CASE WHEN (store_sales#24 > 0.00) THEN (store_sales#36 / store_sales#24) END) +Join condition: (CASE WHEN (web_sales#62 > 0.00) THEN (web_sales#74 / web_sales#62) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) (100) Project [codegen id : 24] -Output [6]: [ca_county#9, d_year#6, (web_sales#60 / web_sales#48) AS web_q1_q2_increase#73, (store_sales#24 / store_sales#12) AS store_q1_q2_increase#74, (web_sales#72 / web_sales#60) AS web_q2_q3_increase#75, (store_sales#36 / store_sales#24) AS store_q2_q3_increase#76] -Input [10]: [ca_county#9, d_year#6, store_sales#12, store_sales#24, store_sales#36, ca_county#45, web_sales#48, web_sales#60, ca_county#69, web_sales#72] +Output [6]: [ca_county#9, d_year#6, (web_sales#62 / web_sales#50) AS web_q1_q2_increase#75, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#76, (web_sales#74 / web_sales#62) AS web_q2_q3_increase#77, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#78] +Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62, ca_county#71, web_sales#74] (101) RowToColumnar -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#73, store_q1_q2_increase#74, web_q2_q3_increase#75, store_q2_q3_increase#76] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] (102) CometColumnarExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#73, store_q1_q2_increase#74, web_q2_q3_increase#75, store_q2_q3_increase#76] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] (103) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#73, store_q1_q2_increase#74, web_q2_q3_increase#75, store_q2_q3_increase#76] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#73, store_q1_q2_increase#74, web_q2_q3_increase#75, store_q2_q3_increase#76], [ca_county#9 ASC NULLS FIRST] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] +Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78], [ca_county#9 ASC NULLS FIRST] (104) ColumnarToRow [codegen id : 25] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#73, store_q1_q2_increase#74, web_q2_q3_increase#75, store_q2_q3_increase#76] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] ===== Subqueries ===== @@ -601,7 +613,7 @@ Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#16 +Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 BroadcastExchange (112) +- * ColumnarToRow (111) +- CometFilter (110) @@ -609,24 +621,24 @@ BroadcastExchange (112) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (110) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) (111) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] (112) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -Subquery:3 Hosting operator id = 34 Hosting Expression = ss_sold_date_sk#27 IN dynamicpruning#28 +Subquery:3 Hosting operator id = 34 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (116) +- * ColumnarToRow (115) +- CometFilter (114) @@ -634,27 +646,27 @@ BroadcastExchange (116) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_qoy#31] +Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (114) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_qoy#31] -Condition : ((((isnotnull(d_qoy#31) AND isnotnull(d_year#30)) AND (d_qoy#31 = 3)) AND (d_year#30 = 2000)) AND isnotnull(d_date_sk#29)) +Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] +Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) (115) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#29, d_year#30, d_qoy#31] +Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] (116) BroadcastExchange -Input [3]: [d_date_sk#29, d_year#30, d_qoy#31] +Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:4 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#16 +Subquery:5 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#17 -Subquery:6 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#28 +Subquery:6 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#29 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index 925a4a7f30..e765d9ae12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -13,9 +13,9 @@ WholeStageCodegen (25) Project [ca_county,d_year,store_sales,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] BroadcastHashJoin [ca_county,ca_county] - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_county,d_qoy,d_year] #2 RowToColumnar WholeStageCodegen (3) @@ -47,9 +47,9 @@ WholeStageCodegen (25) InputAdapter BroadcastExchange #5 WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_county,d_qoy,d_year] #6 RowToColumnar WholeStageCodegen (6) @@ -76,9 +76,9 @@ WholeStageCodegen (25) InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_county,d_qoy,d_year] #9 RowToColumnar WholeStageCodegen (10) @@ -105,9 +105,9 @@ WholeStageCodegen (25) InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_county,d_qoy,d_year] #12 RowToColumnar WholeStageCodegen (14) @@ -128,9 +128,9 @@ WholeStageCodegen (25) InputAdapter BroadcastExchange #13 WholeStageCodegen (19) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_county,d_qoy,d_year] #14 RowToColumnar WholeStageCodegen (18) @@ -151,9 +151,9 @@ WholeStageCodegen (25) InputAdapter BroadcastExchange #15 WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_county,d_qoy,d_year] #16 RowToColumnar WholeStageCodegen (22) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index c9ff159239..30cc61779d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* ColumnarToRow (33) -+- CometHashAggregate (32) +* HashAggregate (33) ++- * ColumnarToRow (32) +- CometColumnarExchange (31) +- RowToColumnar (30) +- * HashAggregate (29) @@ -19,9 +19,9 @@ : : +- CometFilter (5) : : +- CometScan parquet spark_catalog.default.item (4) : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometFilter (21) - : +- CometHashAggregate (20) + : +- * Filter (22) + : +- * HashAggregate (21) + : +- * ColumnarToRow (20) : +- CometColumnarExchange (19) : +- RowToColumnar (18) : +- * HashAggregate (17) @@ -123,66 +123,70 @@ Input [3]: [cs_item_sk#7, sum#14, count#15] Input [3]: [cs_item_sk#7, sum#14, count#15] Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(20) CometHashAggregate +(20) ColumnarToRow [codegen id : 4] +Input [3]: [cs_item_sk#7, sum#14, count#15] + +(21) HashAggregate [codegen id : 4] Input [3]: [cs_item_sk#7, sum#14, count#15] Keys [1]: [cs_item_sk#7] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#16] +Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] -(21) CometFilter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) - -(22) ColumnarToRow [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] +(22) Filter [codegen id : 4] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) (23) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] Right keys [1]: [cs_item_sk#7] Join type: Inner -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#16) +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) (25) Project [codegen id : 6] Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] (26) ReusedExchange [Reuses operator id: 38] -Output [1]: [d_date_sk#17] +Output [1]: [d_date_sk#18] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None (28) Project [codegen id : 6] Output [1]: [cs_ext_discount_amt#2] -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#17] +Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#18] (29) HashAggregate [codegen id : 6] Input [1]: [cs_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#18] -Results [1]: [sum#19] +Aggregate Attributes [1]: [sum#19] +Results [1]: [sum#20] (30) RowToColumnar -Input [1]: [sum#19] +Input [1]: [sum#20] (31) CometColumnarExchange -Input [1]: [sum#19] +Input [1]: [sum#20] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(32) CometHashAggregate -Input [1]: [sum#19] +(32) ColumnarToRow [codegen id : 7] +Input [1]: [sum#20] + +(33) HashAggregate [codegen id : 7] +Input [1]: [sum#20] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] - -(33) ColumnarToRow [codegen id : 7] -Input [1]: [excess discount amount#20] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#21] +Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#21,17,2) AS excess discount amount#22] ===== Subqueries ===== @@ -195,25 +199,25 @@ BroadcastExchange (38) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_date#21] +Output [2]: [d_date_sk#18, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (35) CometFilter -Input [2]: [d_date_sk#17, d_date#21] -Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 2000-01-27)) AND (d_date#21 <= 2000-04-26)) AND isnotnull(d_date_sk#17)) +Input [2]: [d_date_sk#18, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) (36) CometProject -Input [2]: [d_date_sk#17, d_date#21] -Arguments: [d_date_sk#17], [d_date_sk#17] +Input [2]: [d_date_sk#18, d_date#23] +Arguments: [d_date_sk#18], [d_date_sk#18] (37) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] +Input [1]: [d_date_sk#18] (38) BroadcastExchange -Input [1]: [d_date_sk#17] +Input [1]: [d_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index 035779c3f6..a0ed5aafdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [sum] + HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (6) @@ -35,10 +35,10 @@ WholeStageCodegen (7) InputAdapter BroadcastExchange #4 WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [(1.3 * avg(cs_ext_discount_amt))] - CometHashAggregate [cs_item_sk,sum,count] + Filter [(1.3 * avg(cs_ext_discount_amt))] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [cs_item_sk] #5 RowToColumnar WholeStageCodegen (3) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index 3249f9fa01..aa365f08ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -1,71 +1,75 @@ == Physical Plan == -* ColumnarToRow (67) -+- CometTakeOrderedAndProject (66) - +- CometHashAggregate (65) - +- CometColumnarExchange (64) - +- CometHashAggregate (63) - +- CometUnion (62) - :- CometHashAggregate (29) - : +- CometColumnarExchange (28) - : +- RowToColumnar (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - :- CometHashAggregate (45) - : +- CometColumnarExchange (44) - : +- RowToColumnar (43) - : +- * HashAggregate (42) - : +- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (32) - : : : : +- CometFilter (31) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- CometHashAggregate (61) - +- CometColumnarExchange (60) - +- RowToColumnar (59) - +- * HashAggregate (58) - +- * Project (57) - +- * BroadcastHashJoin Inner BuildRight (56) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * ColumnarToRow (48) - : : : +- CometFilter (47) - : : : +- CometScan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (49) - : +- ReusedExchange (52) - +- ReusedExchange (55) +TakeOrderedAndProject (71) ++- * HashAggregate (70) + +- * ColumnarToRow (69) + +- CometColumnarExchange (68) + +- RowToColumnar (67) + +- * HashAggregate (66) + +- Union (65) + :- * HashAggregate (30) + : +- * ColumnarToRow (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- * HashAggregate (47) + : +- * ColumnarToRow (46) + : +- CometColumnarExchange (45) + : +- RowToColumnar (44) + : +- * HashAggregate (43) + : +- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * Project (36) + : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : :- * ColumnarToRow (33) + : : : : +- CometFilter (32) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : : +- ReusedExchange (34) + : : +- ReusedExchange (37) + : +- ReusedExchange (40) + +- * HashAggregate (64) + +- * ColumnarToRow (63) + +- CometColumnarExchange (62) + +- RowToColumnar (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * ColumnarToRow (50) + : : : +- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (unknown) Scan parquet spark_catalog.default.store_sales @@ -83,7 +87,7 @@ Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 72] +(4) ReusedExchange [Reuses operator id: 76] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -198,217 +202,236 @@ Input [2]: [i_manufact_id#10, sum#14] Input [2]: [i_manufact_id#10, sum#14] Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometHashAggregate +(29) ColumnarToRow [codegen id : 6] +Input [2]: [i_manufact_id#10, sum#14] + +(30) HashAggregate [codegen id : 6] Input [2]: [i_manufact_id#10, sum#14] Keys [1]: [i_manufact_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] +Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] +Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -Condition : (isnotnull(cs_bill_addr_sk#15) AND isnotnull(cs_item_sk#16)) +(32) CometFilter +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(32) ColumnarToRow [codegen id : 10] -Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] +(33) ColumnarToRow [codegen id : 11] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -(33) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#20] +(34) ReusedExchange [Reuses operator id: 76] +Output [1]: [d_date_sk#22] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] +(35) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(35) Project [codegen id : 10] -Output [3]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17] -Input [5]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#20] +(36) Project [codegen id : 11] +Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] +Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -(36) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#21] +(37) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#23] -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_addr_sk#15] -Right keys [1]: [ca_address_sk#21] +(38) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_bill_addr_sk#17] +Right keys [1]: [ca_address_sk#23] Join type: Inner Join condition: None -(38) Project [codegen id : 10] -Output [2]: [cs_item_sk#16, cs_ext_sales_price#17] -Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, ca_address_sk#21] +(39) Project [codegen id : 11] +Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] -(39) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#22, i_manufact_id#23] +(40) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#24, i_manufact_id#25] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#16] -Right keys [1]: [i_item_sk#22] +(41) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#24] Join type: Inner Join condition: None -(41) Project [codegen id : 10] -Output [2]: [cs_ext_sales_price#17, i_manufact_id#23] -Input [4]: [cs_item_sk#16, cs_ext_sales_price#17, i_item_sk#22, i_manufact_id#23] +(42) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#19, i_manufact_id#25] +Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_manufact_id#25] -(42) HashAggregate [codegen id : 10] -Input [2]: [cs_ext_sales_price#17, i_manufact_id#23] -Keys [1]: [i_manufact_id#23] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#17))] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_manufact_id#23, sum#25] +(43) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#19, i_manufact_id#25] +Keys [1]: [i_manufact_id#25] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum#26] +Results [2]: [i_manufact_id#25, sum#27] -(43) RowToColumnar -Input [2]: [i_manufact_id#23, sum#25] +(44) RowToColumnar +Input [2]: [i_manufact_id#25, sum#27] -(44) CometColumnarExchange -Input [2]: [i_manufact_id#23, sum#25] -Arguments: hashpartitioning(i_manufact_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(45) CometColumnarExchange +Input [2]: [i_manufact_id#25, sum#27] +Arguments: hashpartitioning(i_manufact_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometHashAggregate -Input [2]: [i_manufact_id#23, sum#25] -Keys [1]: [i_manufact_id#23] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#17))] +(46) ColumnarToRow [codegen id : 12] +Input [2]: [i_manufact_id#25, sum#27] + +(47) HashAggregate [codegen id : 12] +Input [2]: [i_manufact_id#25, sum#27] +Keys [1]: [i_manufact_id#25] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] +Results [2]: [i_manufact_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(47) CometFilter -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) +(49) CometFilter +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) -(48) ColumnarToRow [codegen id : 15] -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +(50) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -(49) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#31] +(51) ReusedExchange [Reuses operator id: 76] +Output [1]: [d_date_sk#35] -(50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#31] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#35] Join type: Inner Join condition: None -(51) Project [codegen id : 15] -Output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] -(52) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#32] +(54) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#36] -(53) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_bill_addr_sk#27] -Right keys [1]: [ca_address_sk#32] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#31] +Right keys [1]: [ca_address_sk#36] Join type: Inner Join condition: None -(54) Project [codegen id : 15] -Output [2]: [ws_item_sk#26, ws_ext_sales_price#28] -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -(55) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#33, i_manufact_id#34] +(57) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#37, i_manufact_id#38] -(56) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#33] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#30] +Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(57) Project [codegen id : 15] -Output [2]: [ws_ext_sales_price#28, i_manufact_id#34] -Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#32, i_manufact_id#38] +Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_manufact_id#38] + +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#32, i_manufact_id#38] +Keys [1]: [i_manufact_id#38] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum#39] +Results [2]: [i_manufact_id#38, sum#40] -(58) HashAggregate [codegen id : 15] -Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] -Keys [1]: [i_manufact_id#34] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] -Aggregate Attributes [1]: [sum#35] -Results [2]: [i_manufact_id#34, sum#36] +(61) RowToColumnar +Input [2]: [i_manufact_id#38, sum#40] -(59) RowToColumnar -Input [2]: [i_manufact_id#34, sum#36] +(62) CometColumnarExchange +Input [2]: [i_manufact_id#38, sum#40] +Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(60) CometColumnarExchange -Input [2]: [i_manufact_id#34, sum#36] -Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(63) ColumnarToRow [codegen id : 18] +Input [2]: [i_manufact_id#38, sum#40] -(61) CometHashAggregate -Input [2]: [i_manufact_id#34, sum#36] -Keys [1]: [i_manufact_id#34] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] +(64) HashAggregate [codegen id : 18] +Input [2]: [i_manufact_id#38, sum#40] +Keys [1]: [i_manufact_id#38] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] +Results [2]: [i_manufact_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] -(62) CometUnion -Child 0 Input [2]: [i_manufact_id#10, total_sales#37] -Child 1 Input [2]: [i_manufact_id#23, total_sales#38] -Child 2 Input [2]: [i_manufact_id#34, total_sales#39] +(65) Union -(63) CometHashAggregate -Input [2]: [i_manufact_id#10, total_sales#37] +(66) HashAggregate [codegen id : 19] +Input [2]: [i_manufact_id#10, total_sales#16] Keys [1]: [i_manufact_id#10] -Functions [1]: [partial_sum(total_sales#37)] +Functions [1]: [partial_sum(total_sales#16)] +Aggregate Attributes [2]: [sum#43, isEmpty#44] +Results [3]: [i_manufact_id#10, sum#45, isEmpty#46] -(64) CometColumnarExchange -Input [3]: [i_manufact_id#10, sum#40, isEmpty#41] +(67) RowToColumnar +Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] + +(68) CometColumnarExchange +Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(65) CometHashAggregate -Input [3]: [i_manufact_id#10, sum#40, isEmpty#41] -Keys [1]: [i_manufact_id#10] -Functions [1]: [sum(total_sales#37)] +(69) ColumnarToRow [codegen id : 20] +Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] -(66) CometTakeOrderedAndProject -Input [2]: [i_manufact_id#10, total_sales#42] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#42 ASC NULLS FIRST], output=[i_manufact_id#10,total_sales#42]), 100, [total_sales#42 ASC NULLS FIRST], [i_manufact_id#10, total_sales#42] +(70) HashAggregate [codegen id : 20] +Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] +Keys [1]: [i_manufact_id#10] +Functions [1]: [sum(total_sales#16)] +Aggregate Attributes [1]: [sum(total_sales#16)#47] +Results [2]: [i_manufact_id#10, sum(total_sales#16)#47 AS total_sales#48] -(67) ColumnarToRow [codegen id : 16] -Input [2]: [i_manufact_id#10, total_sales#42] +(71) TakeOrderedAndProject +Input [2]: [i_manufact_id#10, total_sales#48] +Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#10, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#43, d_moy#44] +Output [3]: [d_date_sk#6, d_year#49, d_moy#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter -Input [3]: [d_date_sk#6, d_year#43, d_moy#44] -Condition : ((((isnotnull(d_year#43) AND isnotnull(d_moy#44)) AND (d_year#43 = 1998)) AND (d_moy#44 = 5)) AND isnotnull(d_date_sk#6)) +(73) CometFilter +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 5)) AND isnotnull(d_date_sk#6)) -(70) CometProject -Input [3]: [d_date_sk#6, d_year#43, d_moy#44] +(74) CometProject +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] Arguments: [d_date_sk#6], [d_date_sk#6] -(71) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(72) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 9dd28b41bf..5735fbbc06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -1,101 +1,113 @@ -WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sales,i_manufact_id] - CometHashAggregate [i_manufact_id,sum,isEmpty] +TakeOrderedAndProject [total_sales,i_manufact_id] + WholeStageCodegen (20) + HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [i_manufact_id] #1 - CometHashAggregate [i_manufact_id,total_sales] - CometUnion - CometHashAggregate [i_manufact_id,sum] - CometColumnarExchange [i_manufact_id] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) + RowToColumnar + WholeStageCodegen (19) + HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_manufact_id,i_manufact_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [i_manufact_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] + WholeStageCodegen (12) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id] #7 + RowToColumnar + WholeStageCodegen (11) + HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow + ReusedExchange [i_item_sk,i_manufact_id] #5 + WholeStageCodegen (18) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id] #8 + RowToColumnar + WholeStageCodegen (17) + HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [i_manufact_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [i_manufact_id,sum] - CometColumnarExchange [i_manufact_id] #7 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 - CometHashAggregate [i_manufact_id,sum] - CometColumnarExchange [i_manufact_id] #8 - RowToColumnar - WholeStageCodegen (15) - HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt index d2ff5ebe98..9e9b7dbf88 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt @@ -5,9 +5,9 @@ +- RowToColumnar (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) - :- * ColumnarToRow (26) - : +- CometFilter (25) - : +- CometHashAggregate (24) + :- * Filter (26) + : +- * HashAggregate (25) + : +- * ColumnarToRow (24) : +- CometColumnarExchange (23) : +- RowToColumnar (22) : +- * HashAggregate (21) @@ -143,59 +143,61 @@ Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometHashAggregate +(24) ColumnarToRow [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] + +(25) HashAggregate [codegen id : 6] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#16] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] -Condition : ((cnt#16 >= 15) AND (cnt#16 <= 20)) - -(26) ColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] +(26) Filter [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) (unknown) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (28) CometFilter -Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] -Condition : isnotnull(c_customer_sk#17) +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Condition : isnotnull(c_customer_sk#18) (29) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] (30) BroadcastExchange -Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#17] +Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None (32) Project [codegen id : 6] -Output [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16, c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] (33) RowToColumnar -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] (34) CometColumnarExchange -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] -Arguments: rangepartitioning(c_last_name#20 ASC NULLS FIRST, c_first_name#19 ASC NULLS FIRST, c_salutation#18 ASC NULLS FIRST, c_preferred_cust_flag#21 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] (35) CometSort -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] -Arguments: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16], [c_last_name#20 ASC NULLS FIRST, c_first_name#19 ASC NULLS FIRST, c_salutation#18 ASC NULLS FIRST, c_preferred_cust_flag#21 DESC NULLS LAST] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST] (36) ColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== @@ -208,18 +210,18 @@ BroadcastExchange (41) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#22, d_dom#23] +Output [3]: [d_date_sk#7, d_year#23, d_dom#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct (38) CometFilter -Input [3]: [d_date_sk#7, d_year#22, d_dom#23] -Condition : (((((d_dom#23 >= 1) AND (d_dom#23 <= 3)) OR ((d_dom#23 >= 25) AND (d_dom#23 <= 28))) AND d_year#22 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) (39) CometProject -Input [3]: [d_date_sk#7, d_year#22, d_dom#23] +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] Arguments: [d_date_sk#7], [d_date_sk#7] (40) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index eed22a0307..0b46527a9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -7,10 +7,10 @@ WholeStageCodegen (7) WholeStageCodegen (6) Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 RowToColumnar WholeStageCodegen (4) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 4d8be2c63a..5205f8b0e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- * ColumnarToRow (42) +- CometColumnarExchange (41) +- RowToColumnar (40) +- * HashAggregate (39) @@ -236,17 +236,19 @@ Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, c Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(42) CometHashAggregate +(42) ColumnarToRow [codegen id : 10] +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] + +(43) HashAggregate [codegen id : 10] Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Functions [10]: [count(1), min(cd_dep_count#23), max(cd_dep_count#23), avg(cd_dep_count#23), min(cd_dep_employed_count#24), max(cd_dep_employed_count#24), avg(cd_dep_employed_count#24), min(cd_dep_college_count#25), max(cd_dep_college_count#25), avg(cd_dep_college_count#25)] +Aggregate Attributes [10]: [count(1)#52, min(cd_dep_count#23)#53, max(cd_dep_count#23)#54, avg(cd_dep_count#23)#55, min(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, avg(cd_dep_employed_count#24)#58, min(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, avg(cd_dep_college_count#25)#61] +Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, count(1)#52 AS cnt1#62, min(cd_dep_count#23)#53 AS min(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, avg(cd_dep_count#23)#55 AS avg(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, min(cd_dep_employed_count#24)#56 AS min(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, avg(cd_dep_employed_count#24)#58 AS avg(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, min(cd_dep_college_count#25)#59 AS min(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, avg(cd_dep_college_count#25)#61 AS avg(cd_dep_college_count)#73, cd_dep_count#23] -(43) CometTakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#52, min(cd_dep_count)#53, max(cd_dep_count)#54, avg(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, min(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, avg(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, min(cd_dep_college_count)#61, max(cd_dep_college_count)#62, avg(cd_dep_college_count)#63, cd_dep_count#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#19 ASC NULLS FIRST,cd_gender#21 ASC NULLS FIRST,cd_marital_status#22 ASC NULLS FIRST,cd_dep_count#23 ASC NULLS FIRST,cd_dep_employed_count#24 ASC NULLS FIRST,cd_dep_college_count#25 ASC NULLS FIRST], output=[ca_state#19,cd_gender#21,cd_marital_status#22,cnt1#52,min(cd_dep_count)#53,max(cd_dep_count)#54,avg(cd_dep_count)#55,cd_dep_employed_count#24,cnt2#56,min(cd_dep_employed_count)#57,max(cd_dep_employed_count)#58,avg(cd_dep_employed_count)#59,cd_dep_college_count#25,cnt3#60,min(cd_dep_college_count)#61,max(cd_dep_college_count)#62,avg(cd_dep_college_count)#63]), 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#52, min(cd_dep_count)#53, max(cd_dep_count)#54, avg(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, min(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, avg(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, min(cd_dep_college_count)#61, max(cd_dep_college_count)#62, avg(cd_dep_college_count)#63] - -(44) ColumnarToRow [codegen id : 10] -Input [17]: [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#52, min(cd_dep_count)#53, max(cd_dep_count)#54, avg(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, min(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, avg(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, min(cd_dep_college_count)#61, max(cd_dep_college_count)#62, avg(cd_dep_college_count)#63] +(44) TakeOrderedAndProject +Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73, cd_dep_count#23] +Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73] ===== Subqueries ===== @@ -259,18 +261,18 @@ BroadcastExchange (49) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#64, d_qoy#65] +Output [3]: [d_date_sk#9, d_year#74, d_qoy#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct (46) CometFilter -Input [3]: [d_date_sk#9, d_year#64, d_qoy#65] -Condition : ((((isnotnull(d_year#64) AND isnotnull(d_qoy#65)) AND (d_year#64 = 2002)) AND (d_qoy#65 < 4)) AND isnotnull(d_date_sk#9)) +Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) (47) CometProject -Input [3]: [d_date_sk#9, d_year#64, d_qoy#65] +Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] Arguments: [d_date_sk#9], [d_date_sk#9] (48) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index c2b0d7dde1..7dd83ba9b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] + WholeStageCodegen (10) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 RowToColumnar WholeStageCodegen (9) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt index a71f17fb59..61f5a576fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt @@ -1,34 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- Window (28) - +- * ColumnarToRow (27) - +- CometSort (26) - +- CometColumnarExchange (25) - +- CometHashAggregate (24) - +- CometColumnarExchange (23) - +- RowToColumnar (22) - +- * HashAggregate (21) - +- * Expand (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store (13) +TakeOrderedAndProject (32) ++- * Project (31) + +- Window (30) + +- * ColumnarToRow (29) + +- CometSort (28) + +- CometColumnarExchange (27) + +- RowToColumnar (26) + +- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometColumnarExchange (23) + +- RowToColumnar (22) + +- * HashAggregate (21) + +- * Expand (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometProject (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.store (13) (unknown) Scan parquet spark_catalog.default.store_sales @@ -46,7 +48,7 @@ Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 35] +(4) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -137,63 +139,71 @@ Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] Arguments: hashpartitioning(i_category#13, i_class#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometHashAggregate +(24) ColumnarToRow [codegen id : 5] +Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] + +(25) HashAggregate [codegen id : 5] Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledValue(ss_ext_sales_price#3))#21] +Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS _w0#24, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS _w1#25, CASE WHEN (cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint) = 0) THEN i_category#13 END AS _w2#26] + +(26) RowToColumnar +Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -(25) CometColumnarExchange -Input [7]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24] -Arguments: hashpartitioning(_w1#23, _w2#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(27) CometColumnarExchange +Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] +Arguments: hashpartitioning(_w1#25, _w2#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) CometSort -Input [7]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24] -Arguments: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24], [_w1#23 ASC NULLS FIRST, _w2#24 ASC NULLS FIRST, _w0#22 ASC NULLS FIRST] +(28) CometSort +Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] +Arguments: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26], [_w1#25 ASC NULLS FIRST, _w2#26 ASC NULLS FIRST, _w0#24 ASC NULLS FIRST] -(27) ColumnarToRow [codegen id : 5] -Input [7]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24] +(29) ColumnarToRow [codegen id : 6] +Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -(28) Window -Input [7]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24] -Arguments: [rank(_w0#22) windowspecdefinition(_w1#23, _w2#24, _w0#22 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#25], [_w1#23, _w2#24], [_w0#22 ASC NULLS FIRST] +(30) Window +Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] +Arguments: [rank(_w0#24) windowspecdefinition(_w1#25, _w2#26, _w0#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#27], [_w1#25, _w2#26], [_w0#24 ASC NULLS FIRST] -(29) Project [codegen id : 6] -Output [5]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, rank_within_parent#25] -Input [8]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, _w0#22, _w1#23, _w2#24, rank_within_parent#25] +(31) Project [codegen id : 7] +Output [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] +Input [8]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26, rank_within_parent#27] -(30) TakeOrderedAndProject -Input [5]: [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, rank_within_parent#25] -Arguments: 100, [lochierarchy#21 DESC NULLS LAST, CASE WHEN (lochierarchy#21 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#25 ASC NULLS FIRST], [gross_margin#20, i_category#13, i_class#14, lochierarchy#21, rank_within_parent#25] +(32) TakeOrderedAndProject +Input [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] +Arguments: 100, [lochierarchy#23 DESC NULLS LAST, CASE WHEN (lochierarchy#23 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#27 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#26] +Output [2]: [d_date_sk#7, d_year#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter -Input [2]: [d_date_sk#7, d_year#26] -Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#7)) +(34) CometFilter +Input [2]: [d_date_sk#7, d_year#28] +Condition : ((isnotnull(d_year#28) AND (d_year#28 = 2001)) AND isnotnull(d_date_sk#7)) -(33) CometProject -Input [2]: [d_date_sk#7, d_year#26] +(35) CometProject +Input [2]: [d_date_sk#7, d_year#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(34) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(35) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index e9e6131a9e..f55a126f07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -1,51 +1,55 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (6) + WholeStageCodegen (7) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (5) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometSort [_w1,_w2,_w0] CometColumnarExchange [_w1,_w2] #1 - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum] - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] - Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] + Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index d42b2f1bf6..50b19fe4ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* ColumnarToRow (55) -+- CometHashAggregate (54) +* HashAggregate (55) ++- * ColumnarToRow (54) +- CometColumnarExchange (53) +- RowToColumnar (52) +- * HashAggregate (51) @@ -298,13 +298,15 @@ Input [1]: [count#26] Input [1]: [count#26] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(54) CometHashAggregate +(54) ColumnarToRow [codegen id : 13] +Input [1]: [count#26] + +(55) HashAggregate [codegen id : 13] Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] - -(55) ColumnarToRow [codegen id : 13] -Input [1]: [count(1)#27] +Aggregate Attributes [1]: [count(1)#27] +Results [1]: [count(1)#27 AS count(1)#28] ===== Subqueries ===== @@ -317,18 +319,18 @@ BroadcastExchange (60) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#28] +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct (57) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#28] -Condition : (((isnotnull(d_month_seq#28) AND (d_month_seq#28 >= 1200)) AND (d_month_seq#28 <= 1211)) AND isnotnull(d_date_sk#4)) +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) (58) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#28] +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] (59) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index 28771f9d42..c3e01e4f26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (13) - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),count(1),count] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (12) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index 092603e845..1866b9e21a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -9,9 +9,9 @@ TakeOrderedAndProject (120) : : :- * Project (58) : : : +- * BroadcastHashJoin Inner BuildRight (57) : : : :- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * ColumnarToRow (18) - : : : : : +- CometFilter (17) - : : : : : +- CometHashAggregate (16) + : : : : :- * Filter (18) + : : : : : +- * HashAggregate (17) + : : : : : +- * ColumnarToRow (16) : : : : : +- CometColumnarExchange (15) : : : : : +- RowToColumnar (14) : : : : : +- * HashAggregate (13) @@ -28,8 +28,8 @@ TakeOrderedAndProject (120) : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) : : : : : +- ReusedExchange (10) : : : : +- BroadcastExchange (36) - : : : : +- * ColumnarToRow (35) - : : : : +- CometHashAggregate (34) + : : : : +- * HashAggregate (35) + : : : : +- * ColumnarToRow (34) : : : : +- CometColumnarExchange (33) : : : : +- RowToColumnar (32) : : : : +- * HashAggregate (31) @@ -46,9 +46,9 @@ TakeOrderedAndProject (120) : : : : : +- CometScan parquet spark_catalog.default.store_sales (22) : : : : +- ReusedExchange (28) : : : +- BroadcastExchange (56) - : : : +- * ColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometHashAggregate (53) + : : : +- * Filter (55) + : : : +- * HashAggregate (54) + : : : +- * ColumnarToRow (53) : : : +- CometColumnarExchange (52) : : : +- RowToColumnar (51) : : : +- * HashAggregate (50) @@ -65,8 +65,8 @@ TakeOrderedAndProject (120) : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) : : : +- ReusedExchange (47) : : +- BroadcastExchange (76) - : : +- * ColumnarToRow (75) - : : +- CometHashAggregate (74) + : : +- * HashAggregate (75) + : : +- * ColumnarToRow (74) : : +- CometColumnarExchange (73) : : +- RowToColumnar (72) : : +- * HashAggregate (71) @@ -83,9 +83,9 @@ TakeOrderedAndProject (120) : : : +- CometScan parquet spark_catalog.default.catalog_sales (62) : : +- ReusedExchange (68) : +- BroadcastExchange (97) - : +- * ColumnarToRow (96) - : +- CometFilter (95) - : +- CometHashAggregate (94) + : +- * Filter (96) + : +- * HashAggregate (95) + : +- * ColumnarToRow (94) : +- CometColumnarExchange (93) : +- RowToColumnar (92) : +- * HashAggregate (91) @@ -102,8 +102,8 @@ TakeOrderedAndProject (120) : : +- CometScan parquet spark_catalog.default.web_sales (82) : +- ReusedExchange (88) +- BroadcastExchange (117) - +- * ColumnarToRow (116) - +- CometHashAggregate (115) + +- * HashAggregate (116) + +- * ColumnarToRow (115) +- CometColumnarExchange (114) +- RowToColumnar (113) +- * HashAggregate (112) @@ -191,485 +191,497 @@ Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_fl Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 24] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] + +(17) HashAggregate [codegen id : 24] Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] Functions [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))] +Aggregate Attributes [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22] +Results [2]: [c_customer_id#2 AS customer_id#23, sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22 AS year_total#24] -(17) CometFilter -Input [2]: [customer_id#22, year_total#23] -Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.000000)) - -(18) ColumnarToRow [codegen id : 24] -Input [2]: [customer_id#22, year_total#23] +(18) Filter [codegen id : 24] +Input [2]: [customer_id#23, year_total#24] +Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.000000)) (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (20) CometFilter -Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) +Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_customer_id#26)) (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] (unknown) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] +Output [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#37), dynamicpruningexpression(ss_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (23) CometFilter -Input [6]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] -Condition : isnotnull(ss_customer_sk#32) +Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Condition : isnotnull(ss_customer_sk#33) (24) ColumnarToRow [codegen id : 4] -Input [6]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] +Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] (25) BroadcastExchange -Input [6]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] +Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#24] -Right keys [1]: [ss_customer_sk#32] +Left keys [1]: [c_customer_sk#25] +Right keys [1]: [ss_customer_sk#33] Join type: Inner Join condition: None (27) Project [codegen id : 6] -Output [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] -Input [14]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37] +Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Input [14]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] (28) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#39, d_year#40] +Output [2]: [d_date_sk#40, d_year#41] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#37] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [ss_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None (30) Project [codegen id : 6] -Output [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, d_year#40] -Input [14]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, ss_sold_date_sk#37, d_date_sk#39, d_year#40] +Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] +Input [14]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] (31) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_sales_price#34, ss_ext_wholesale_cost#35, ss_ext_list_price#36, d_year#40] -Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40] -Functions [1]: [partial_sum(((((ss_ext_list_price#36 - ss_ext_wholesale_cost#35) - ss_ext_discount_amt#33) + ss_ext_sales_price#34) / 2))] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40, sum#43, isEmpty#44] +Input [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] +Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] +Functions [1]: [partial_sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] (32) RowToColumnar -Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40, sum#43, isEmpty#44] +Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] (33) CometColumnarExchange -Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] +Arguments: hashpartitioning(c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometHashAggregate -Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40, sum#43, isEmpty#44] -Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, d_year#40] -Functions [1]: [sum(((((ss_ext_list_price#36 - ss_ext_wholesale_cost#35) - ss_ext_discount_amt#33) + ss_ext_sales_price#34) / 2))] +(34) ColumnarToRow [codegen id : 7] +Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] -(35) ColumnarToRow [codegen id : 7] -Input [8]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#52] +(35) HashAggregate [codegen id : 7] +Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] +Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] +Functions [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] +Aggregate Attributes [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22] +Results [8]: [c_customer_id#26 AS customer_id#46, c_first_name#27 AS customer_first_name#47, c_last_name#28 AS customer_last_name#48, c_preferred_cust_flag#29 AS customer_preferred_cust_flag#49, c_birth_country#30 AS customer_birth_country#50, c_login#31 AS customer_login#51, c_email_address#32 AS customer_email_address#52, sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22 AS year_total#53] (36) BroadcastExchange -Input [8]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#52] +Input [8]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] (37) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#45] +Left keys [1]: [customer_id#23] +Right keys [1]: [customer_id#46] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] +Output [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (39) CometFilter -Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] -Condition : (isnotnull(c_customer_sk#53) AND isnotnull(c_customer_id#54)) +Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] +Condition : (isnotnull(c_customer_sk#54) AND isnotnull(c_customer_id#55)) (40) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] +Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] +Output [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(cs_sold_date_sk#67), dynamicpruningexpression(cs_sold_date_sk#67 IN dynamicpruning#68)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (42) CometFilter -Input [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] -Condition : isnotnull(cs_bill_customer_sk#61) +Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Condition : isnotnull(cs_bill_customer_sk#62) (43) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] +Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] (44) BroadcastExchange -Input [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] +Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] (45) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#53] -Right keys [1]: [cs_bill_customer_sk#61] +Left keys [1]: [c_customer_sk#54] +Right keys [1]: [cs_bill_customer_sk#62] Join type: Inner Join condition: None (46) Project [codegen id : 10] -Output [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] -Input [14]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] +Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Input [14]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] (47) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#68, d_year#69] +Output [2]: [d_date_sk#69, d_year#70] (48) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#66] -Right keys [1]: [d_date_sk#68] +Left keys [1]: [cs_sold_date_sk#67] +Right keys [1]: [d_date_sk#69] Join type: Inner Join condition: None (49) Project [codegen id : 10] -Output [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, d_year#69] -Input [14]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66, d_date_sk#68, d_year#69] +Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] +Input [14]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67, d_date_sk#69, d_year#70] (50) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, d_year#69] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69] -Functions [1]: [partial_sum(((((cs_ext_list_price#65 - cs_ext_wholesale_cost#64) - cs_ext_discount_amt#62) + cs_ext_sales_price#63) / 2))] -Aggregate Attributes [2]: [sum#70, isEmpty#71] -Results [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69, sum#72, isEmpty#73] +Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] +Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] +Functions [1]: [partial_sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] +Aggregate Attributes [2]: [sum#71, isEmpty#72] +Results [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] (51) RowToColumnar -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69, sum#72, isEmpty#73] +Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] (52) CometColumnarExchange -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69, sum#72, isEmpty#73] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] +Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69, sum#72, isEmpty#73] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60, d_year#69] -Functions [1]: [sum(((((cs_ext_list_price#65 - cs_ext_wholesale_cost#64) - cs_ext_discount_amt#62) + cs_ext_sales_price#63) / 2))] +(53) ColumnarToRow [codegen id : 11] +Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] -(54) CometFilter -Input [2]: [customer_id#74, year_total#75] -Condition : (isnotnull(year_total#75) AND (year_total#75 > 0.000000)) +(54) HashAggregate [codegen id : 11] +Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] +Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] +Functions [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] +Aggregate Attributes [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75] +Results [2]: [c_customer_id#55 AS customer_id#76, sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75 AS year_total#77] -(55) ColumnarToRow [codegen id : 11] -Input [2]: [customer_id#74, year_total#75] +(55) Filter [codegen id : 11] +Input [2]: [customer_id#76, year_total#77] +Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.000000)) (56) BroadcastExchange -Input [2]: [customer_id#74, year_total#75] +Input [2]: [customer_id#76, year_total#77] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] (57) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#74] +Left keys [1]: [customer_id#23] +Right keys [1]: [customer_id#76] Join type: Inner Join condition: None (58) Project [codegen id : 24] -Output [11]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#52, year_total#75] -Input [12]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#52, customer_id#74, year_total#75] +Output [11]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77] +Input [12]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, customer_id#76, year_total#77] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] +Output [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (60) CometFilter -Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] -Condition : (isnotnull(c_customer_sk#76) AND isnotnull(c_customer_id#77)) +Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] +Condition : (isnotnull(c_customer_sk#78) AND isnotnull(c_customer_id#79)) (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] +Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#84, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] +Output [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#89), dynamicpruningexpression(cs_sold_date_sk#89 IN dynamicpruning#90)] +PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#92)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (63) CometFilter -Input [6]: [cs_bill_customer_sk#84, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] -Condition : isnotnull(cs_bill_customer_sk#84) +Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +Condition : isnotnull(cs_bill_customer_sk#86) (64) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#84, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] +Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] (65) BroadcastExchange -Input [6]: [cs_bill_customer_sk#84, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] +Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] (66) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#76] -Right keys [1]: [cs_bill_customer_sk#84] +Left keys [1]: [c_customer_sk#78] +Right keys [1]: [cs_bill_customer_sk#86] Join type: Inner Join condition: None (67) Project [codegen id : 14] -Output [12]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] -Input [14]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, cs_bill_customer_sk#84, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89] +Output [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +Input [14]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] (68) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#91, d_year#92] +Output [2]: [d_date_sk#93, d_year#94] (69) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#89] -Right keys [1]: [d_date_sk#91] +Left keys [1]: [cs_sold_date_sk#91] +Right keys [1]: [d_date_sk#93] Join type: Inner Join condition: None (70) Project [codegen id : 14] -Output [12]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, d_year#92] -Input [14]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, cs_sold_date_sk#89, d_date_sk#91, d_year#92] +Output [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] +Input [14]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91, d_date_sk#93, d_year#94] (71) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, cs_ext_discount_amt#85, cs_ext_sales_price#86, cs_ext_wholesale_cost#87, cs_ext_list_price#88, d_year#92] -Keys [8]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92] -Functions [1]: [partial_sum(((((cs_ext_list_price#88 - cs_ext_wholesale_cost#87) - cs_ext_discount_amt#85) + cs_ext_sales_price#86) / 2))] -Aggregate Attributes [2]: [sum#93, isEmpty#94] -Results [10]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92, sum#95, isEmpty#96] +Input [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] +Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94] +Functions [1]: [partial_sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))] +Aggregate Attributes [2]: [sum#95, isEmpty#96] +Results [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] (72) RowToColumnar -Input [10]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92, sum#95, isEmpty#96] +Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] (73) CometColumnarExchange -Input [10]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92, sum#95, isEmpty#96] -Arguments: hashpartitioning(c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] +Arguments: hashpartitioning(c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometHashAggregate -Input [10]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92, sum#95, isEmpty#96] -Keys [8]: [c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83, d_year#92] -Functions [1]: [sum(((((cs_ext_list_price#88 - cs_ext_wholesale_cost#87) - cs_ext_discount_amt#85) + cs_ext_sales_price#86) / 2))] +(74) ColumnarToRow [codegen id : 15] +Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] -(75) ColumnarToRow [codegen id : 15] -Input [2]: [customer_id#97, year_total#98] +(75) HashAggregate [codegen id : 15] +Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] +Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94] +Functions [1]: [sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))] +Aggregate Attributes [1]: [sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))#75] +Results [2]: [c_customer_id#79 AS customer_id#99, sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))#75 AS year_total#100] (76) BroadcastExchange -Input [2]: [customer_id#97, year_total#98] +Input [2]: [customer_id#99, year_total#100] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] (77) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#97] +Left keys [1]: [customer_id#23] +Right keys [1]: [customer_id#99] Join type: Inner -Join condition: (CASE WHEN (year_total#75 > 0.000000) THEN (year_total#98 / year_total#75) END > CASE WHEN (year_total#23 > 0.000000) THEN (year_total#52 / year_total#23) END) +Join condition: (CASE WHEN (year_total#77 > 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END) (78) Project [codegen id : 24] -Output [10]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#75, year_total#98] -Input [13]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#52, year_total#75, customer_id#97, year_total#98] +Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100] +Input [13]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77, customer_id#99, year_total#100] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] +Output [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (80) CometFilter -Input [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] -Condition : (isnotnull(c_customer_sk#99) AND isnotnull(c_customer_id#100)) +Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Condition : (isnotnull(c_customer_sk#101) AND isnotnull(c_customer_id#102)) (81) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] +Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] (unknown) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] +Output [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_sold_date_sk#112 IN dynamicpruning#113)] +PartitionFilters: [isnotnull(ws_sold_date_sk#114), dynamicpruningexpression(ws_sold_date_sk#114 IN dynamicpruning#115)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (83) CometFilter -Input [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] -Condition : isnotnull(ws_bill_customer_sk#107) +Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +Condition : isnotnull(ws_bill_customer_sk#109) (84) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] +Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] (85) BroadcastExchange -Input [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] +Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] (86) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#99] -Right keys [1]: [ws_bill_customer_sk#107] +Left keys [1]: [c_customer_sk#101] +Right keys [1]: [ws_bill_customer_sk#109] Join type: Inner Join condition: None (87) Project [codegen id : 18] -Output [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] -Input [14]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] +Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +Input [14]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] (88) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#114, d_year#115] +Output [2]: [d_date_sk#116, d_year#117] (89) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#112] -Right keys [1]: [d_date_sk#114] +Left keys [1]: [ws_sold_date_sk#114] +Right keys [1]: [d_date_sk#116] Join type: Inner Join condition: None (90) Project [codegen id : 18] -Output [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, d_year#115] -Input [14]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112, d_date_sk#114, d_year#115] +Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] +Input [14]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114, d_date_sk#116, d_year#117] (91) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, d_year#115] -Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115] -Functions [1]: [partial_sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))] -Aggregate Attributes [2]: [sum#116, isEmpty#117] -Results [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115, sum#118, isEmpty#119] +Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] +Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] +Functions [1]: [partial_sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] +Aggregate Attributes [2]: [sum#118, isEmpty#119] +Results [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] (92) RowToColumnar -Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115, sum#118, isEmpty#119] +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] (93) CometColumnarExchange -Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115, sum#118, isEmpty#119] -Arguments: hashpartitioning(c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] +Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(94) CometHashAggregate -Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115, sum#118, isEmpty#119] -Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#115] -Functions [1]: [sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))] +(94) ColumnarToRow [codegen id : 19] +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] -(95) CometFilter -Input [2]: [customer_id#120, year_total#121] -Condition : (isnotnull(year_total#121) AND (year_total#121 > 0.000000)) +(95) HashAggregate [codegen id : 19] +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] +Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] +Functions [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] +Aggregate Attributes [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122] +Results [2]: [c_customer_id#102 AS customer_id#123, sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122 AS year_total#124] -(96) ColumnarToRow [codegen id : 19] -Input [2]: [customer_id#120, year_total#121] +(96) Filter [codegen id : 19] +Input [2]: [customer_id#123, year_total#124] +Condition : (isnotnull(year_total#124) AND (year_total#124 > 0.000000)) (97) BroadcastExchange -Input [2]: [customer_id#120, year_total#121] +Input [2]: [customer_id#123, year_total#124] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] (98) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#120] +Left keys [1]: [customer_id#23] +Right keys [1]: [customer_id#123] Join type: Inner Join condition: None (99) Project [codegen id : 24] -Output [11]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#75, year_total#98, year_total#121] -Input [12]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#75, year_total#98, customer_id#120, year_total#121] +Output [11]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124] +Input [12]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, customer_id#123, year_total#124] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129] +Output [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (101) CometFilter -Input [8]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129] -Condition : (isnotnull(c_customer_sk#122) AND isnotnull(c_customer_id#123)) +Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +Condition : (isnotnull(c_customer_sk#125) AND isnotnull(c_customer_id#126)) (102) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129] +Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] (unknown) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] +Output [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#135), dynamicpruningexpression(ws_sold_date_sk#135 IN dynamicpruning#136)] +PartitionFilters: [isnotnull(ws_sold_date_sk#138), dynamicpruningexpression(ws_sold_date_sk#138 IN dynamicpruning#139)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (104) CometFilter -Input [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] -Condition : isnotnull(ws_bill_customer_sk#130) +Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +Condition : isnotnull(ws_bill_customer_sk#133) (105) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] +Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] (106) BroadcastExchange -Input [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] +Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] (107) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#122] -Right keys [1]: [ws_bill_customer_sk#130] +Left keys [1]: [c_customer_sk#125] +Right keys [1]: [ws_bill_customer_sk#133] Join type: Inner Join condition: None (108) Project [codegen id : 22] -Output [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] -Input [14]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] +Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +Input [14]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] (109) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#137, d_year#138] +Output [2]: [d_date_sk#140, d_year#141] (110) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#135] -Right keys [1]: [d_date_sk#137] +Left keys [1]: [ws_sold_date_sk#138] +Right keys [1]: [d_date_sk#140] Join type: Inner Join condition: None (111) Project [codegen id : 22] -Output [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, d_year#138] -Input [14]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135, d_date_sk#137, d_year#138] +Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] +Input [14]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138, d_date_sk#140, d_year#141] (112) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, d_year#138] -Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138] -Functions [1]: [partial_sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))] -Aggregate Attributes [2]: [sum#139, isEmpty#140] -Results [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138, sum#141, isEmpty#142] +Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] +Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] +Functions [1]: [partial_sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] +Aggregate Attributes [2]: [sum#142, isEmpty#143] +Results [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] (113) RowToColumnar -Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138, sum#141, isEmpty#142] +Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] (114) CometColumnarExchange -Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138, sum#141, isEmpty#142] -Arguments: hashpartitioning(c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] +Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] +Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(115) CometHashAggregate -Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138, sum#141, isEmpty#142] -Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#138] -Functions [1]: [sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))] +(115) ColumnarToRow [codegen id : 23] +Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] -(116) ColumnarToRow [codegen id : 23] -Input [2]: [customer_id#143, year_total#144] +(116) HashAggregate [codegen id : 23] +Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] +Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] +Functions [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] +Aggregate Attributes [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122] +Results [2]: [c_customer_id#126 AS customer_id#146, sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122 AS year_total#147] (117) BroadcastExchange -Input [2]: [customer_id#143, year_total#144] +Input [2]: [customer_id#146, year_total#147] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] (118) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#143] +Left keys [1]: [customer_id#23] +Right keys [1]: [customer_id#146] Join type: Inner -Join condition: (CASE WHEN (year_total#75 > 0.000000) THEN (year_total#98 / year_total#75) END > CASE WHEN (year_total#121 > 0.000000) THEN (year_total#144 / year_total#121) END) +Join condition: (CASE WHEN (year_total#77 > 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#124 > 0.000000) THEN (year_total#147 / year_total#124) END) (119) Project [codegen id : 24] -Output [7]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51] -Input [13]: [customer_id#22, customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51, year_total#75, year_total#98, year_total#121, customer_id#143, year_total#144] +Output [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] +Input [13]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124, customer_id#146, year_total#147] (120) TakeOrderedAndProject -Input [7]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51] -Arguments: 100, [customer_id#45 ASC NULLS FIRST, customer_first_name#46 ASC NULLS FIRST, customer_last_name#47 ASC NULLS FIRST, customer_preferred_cust_flag#48 ASC NULLS FIRST, customer_birth_country#49 ASC NULLS FIRST, customer_login#50 ASC NULLS FIRST, customer_email_address#51 ASC NULLS FIRST], [customer_id#45, customer_first_name#46, customer_last_name#47, customer_preferred_cust_flag#48, customer_birth_country#49, customer_login#50, customer_email_address#51] +Input [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] +Arguments: 100, [customer_id#46 ASC NULLS FIRST, customer_first_name#47 ASC NULLS FIRST, customer_last_name#48 ASC NULLS FIRST, customer_preferred_cust_flag#49 ASC NULLS FIRST, customer_birth_country#50 ASC NULLS FIRST, customer_login#51 ASC NULLS FIRST, customer_email_address#52 ASC NULLS FIRST], [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] ===== Subqueries ===== @@ -698,7 +710,7 @@ Input [2]: [d_date_sk#16, d_year#17] Input [2]: [d_date_sk#16, d_year#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#37 IN dynamicpruning#38 +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 BroadcastExchange (128) +- * ColumnarToRow (127) +- CometFilter (126) @@ -706,29 +718,29 @@ BroadcastExchange (128) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#39, d_year#40] +Output [2]: [d_date_sk#40, d_year#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (126) CometFilter -Input [2]: [d_date_sk#39, d_year#40] -Condition : ((isnotnull(d_year#40) AND (d_year#40 = 2002)) AND isnotnull(d_date_sk#39)) +Input [2]: [d_date_sk#40, d_year#41] +Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) (127) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#39, d_year#40] +Input [2]: [d_date_sk#40, d_year#41] (128) BroadcastExchange -Input [2]: [d_date_sk#39, d_year#40] +Input [2]: [d_date_sk#40, d_year#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] -Subquery:3 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 62 Hosting Expression = cs_sold_date_sk#89 IN dynamicpruning#38 +Subquery:4 Hosting operator id = 62 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#39 -Subquery:5 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#114 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 103 Hosting Expression = ws_sold_date_sk#135 IN dynamicpruning#38 +Subquery:6 Hosting operator id = 103 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#39 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 3299726c59..efa385b428 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -9,10 +9,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] - ColumnarToRow - InputAdapter - CometFilter [year_total] - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 RowToColumnar WholeStageCodegen (3) @@ -44,9 +44,9 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 RowToColumnar WholeStageCodegen (6) @@ -78,10 +78,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [year_total] - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 RowToColumnar WholeStageCodegen (10) @@ -107,9 +107,9 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 RowToColumnar WholeStageCodegen (14) @@ -135,10 +135,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter BroadcastExchange #14 WholeStageCodegen (19) - ColumnarToRow - InputAdapter - CometFilter [year_total] - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 RowToColumnar WholeStageCodegen (18) @@ -164,9 +164,9 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter BroadcastExchange #17 WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 RowToColumnar WholeStageCodegen (22) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt index 89bb2af3e9..0a369db73d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (34) -+- CometTakeOrderedAndProject (33) - +- CometHashAggregate (32) +TakeOrderedAndProject (34) ++- * HashAggregate (33) + +- * ColumnarToRow (32) +- CometColumnarExchange (31) +- RowToColumnar (30) +- * HashAggregate (29) @@ -177,17 +177,19 @@ Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) CometHashAggregate +(32) ColumnarToRow [codegen id : 5] +Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] + +(33) HashAggregate [codegen id : 5] Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [2]: [w_state#12, i_item_id#14] Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] +Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27] +Results [4]: [w_state#12, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26 AS sales_before#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27 AS sales_after#29] -(33) CometTakeOrderedAndProject -Input [4]: [w_state#12, i_item_id#14, sales_before#26, sales_after#27] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#12 ASC NULLS FIRST,i_item_id#14 ASC NULLS FIRST], output=[w_state#12,i_item_id#14,sales_before#26,sales_after#27]), 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#26, sales_after#27] - -(34) ColumnarToRow [codegen id : 5] -Input [4]: [w_state#12, i_item_id#14, sales_before#26, sales_after#27] +(34) TakeOrderedAndProject +Input [4]: [w_state#12, i_item_id#14, sales_before#28, sales_after#29] +Arguments: 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#28, sales_after#29] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index 541de7ec5e..cb386096a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - CometHashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] +TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] + WholeStageCodegen (5) + HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [w_state,i_item_id] #1 RowToColumnar WholeStageCodegen (4) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt index 647bf61f60..112e87bb78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometHashAggregate (21) +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- * ColumnarToRow (21) +- CometColumnarExchange (20) +- RowToColumnar (19) +- * HashAggregate (18) @@ -117,15 +117,17 @@ Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometHashAggregate +(21) ColumnarToRow [codegen id : 4] +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] + +(22) HashAggregate [codegen id : 4] Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] Keys [3]: [d_year#2, i_category_id#8, i_category#9] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] +Results [4]: [d_year#2, i_category_id#8, i_category#9, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum(ss_ext_sales_price)#14] -(22) CometTakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#13] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#9 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#9,sum(ss_ext_sales_price)#13]), 100, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#13] - -(23) ColumnarToRow [codegen id : 4] -Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#13] +(23) TakeOrderedAndProject +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] +Arguments: 100, [sum(ss_ext_sales_price)#14 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt index d33229197b..52aa2dbd5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] - CometHashAggregate [d_year,i_category_id,i_category,sum] +TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] + WholeStageCodegen (4) + HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] + ColumnarToRow + InputAdapter CometColumnarExchange [d_year,i_category_id,i_category] #1 RowToColumnar WholeStageCodegen (3) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt index 4e1468e50c..1a832bd08b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometHashAggregate (21) +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- * ColumnarToRow (21) +- CometColumnarExchange (20) +- RowToColumnar (19) +- * HashAggregate (18) @@ -117,15 +117,17 @@ Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22 Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] Arguments: hashpartitioning(s_store_name#9, s_store_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometHashAggregate +(21) ColumnarToRow [codegen id : 4] +Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] + +(22) HashAggregate [codegen id : 4] Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [2]: [s_store_name#9, s_store_id#8] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31] +Results [9]: [s_store_name#9, s_store_id#8, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25,17,2) AS sun_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26,17,2) AS mon_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27,17,2) AS tue_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28,17,2) AS wed_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29,17,2) AS thu_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30,17,2) AS fri_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31,17,2) AS sat_sales#38] -(22) CometTakeOrderedAndProject -Input [9]: [s_store_name#9, s_store_id#8, sun_sales#25, mon_sales#26, tue_sales#27, wed_sales#28, thu_sales#29, fri_sales#30, sat_sales#31] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST,s_store_id#8 ASC NULLS FIRST,sun_sales#25 ASC NULLS FIRST,mon_sales#26 ASC NULLS FIRST,tue_sales#27 ASC NULLS FIRST,wed_sales#28 ASC NULLS FIRST,thu_sales#29 ASC NULLS FIRST,fri_sales#30 ASC NULLS FIRST,sat_sales#31 ASC NULLS FIRST], output=[s_store_name#9,s_store_id#8,sun_sales#25,mon_sales#26,tue_sales#27,wed_sales#28,thu_sales#29,fri_sales#30,sat_sales#31]), 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#25 ASC NULLS FIRST, mon_sales#26 ASC NULLS FIRST, tue_sales#27 ASC NULLS FIRST, wed_sales#28 ASC NULLS FIRST, thu_sales#29 ASC NULLS FIRST, fri_sales#30 ASC NULLS FIRST, sat_sales#31 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#25, mon_sales#26, tue_sales#27, wed_sales#28, thu_sales#29, fri_sales#30, sat_sales#31] - -(23) ColumnarToRow [codegen id : 4] -Input [9]: [s_store_name#9, s_store_id#8, sun_sales#25, mon_sales#26, tue_sales#27, wed_sales#28, thu_sales#29, fri_sales#30, sat_sales#31] +(23) TakeOrderedAndProject +Input [9]: [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] +Arguments: 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#32 ASC NULLS FIRST, mon_sales#33 ASC NULLS FIRST, tue_sales#34 ASC NULLS FIRST, wed_sales#35 ASC NULLS FIRST, thu_sales#36 ASC NULLS FIRST, fri_sales#37 ASC NULLS FIRST, sat_sales#38 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt index 0980f4764e..2fc9968cc0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] +TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + WholeStageCodegen (4) + HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [s_store_name,s_store_id] #1 RowToColumnar WholeStageCodegen (3) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index ac8a1f4f7a..d965422219 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (38) -+- CometTakeOrderedAndProject (37) - +- CometHashAggregate (36) +TakeOrderedAndProject (38) ++- * HashAggregate (37) + +- * ColumnarToRow (36) +- CometColumnarExchange (35) +- RowToColumnar (34) +- * HashAggregate (33) @@ -201,17 +201,19 @@ Input [3]: [ca_zip#11, ca_city#10, sum#18] Input [3]: [ca_zip#11, ca_city#10, sum#18] Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(36) CometHashAggregate +(36) ColumnarToRow [codegen id : 7] +Input [3]: [ca_zip#11, ca_city#10, sum#18] + +(37) HashAggregate [codegen id : 7] Input [3]: [ca_zip#11, ca_city#10, sum#18] Keys [2]: [ca_zip#11, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#19] +Results [3]: [ca_zip#11, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#19,17,2) AS sum(ws_sales_price)#20] -(37) CometTakeOrderedAndProject -Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#19] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#11 ASC NULLS FIRST,ca_city#10 ASC NULLS FIRST], output=[ca_zip#11,ca_city#10,sum(ws_sales_price)#19]), 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#11, ca_city#10, sum(ws_sales_price)#19] - -(38) ColumnarToRow [codegen id : 7] -Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#19] +(38) TakeOrderedAndProject +Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] +Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] ===== Subqueries ===== @@ -224,18 +226,18 @@ BroadcastExchange (43) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#20, d_qoy#21] +Output [3]: [d_date_sk#12, d_year#21, d_qoy#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (40) CometFilter -Input [3]: [d_date_sk#12, d_year#20, d_qoy#21] -Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 2)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#12)) +Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] +Condition : ((((isnotnull(d_qoy#22) AND isnotnull(d_year#21)) AND (d_qoy#22 = 2)) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#12)) (41) CometProject -Input [3]: [d_date_sk#12, d_year#20, d_qoy#21] +Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] Arguments: [d_date_sk#12], [d_date_sk#12] (42) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index b4965dae7c..05022ce60c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - CometHashAggregate [ca_zip,ca_city,sum] +TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] + WholeStageCodegen (7) + HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_zip,ca_city] #1 RowToColumnar WholeStageCodegen (6) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index a5c06c49ae..3fb39c6df1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -4,8 +4,8 @@ TakeOrderedAndProject (41) +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (37) : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * ColumnarToRow (31) - : : +- CometHashAggregate (30) + : :- * HashAggregate (31) + : : +- * ColumnarToRow (30) : : +- CometColumnarExchange (29) : : +- RowToColumnar (28) : : +- * HashAggregate (27) @@ -176,58 +176,60 @@ Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum# Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometHashAggregate +(30) ColumnarToRow [codegen id : 8] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] + +(31) HashAggregate [codegen id : 8] Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(31) ColumnarToRow [codegen id : 8] -Input [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#22, amt#23, profit#24] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#22, sum(UnscaledValue(ss_net_profit#7))#23] +Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#17 AS bought_city#24, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#22,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#23,17,2) AS profit#26] (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28] +Output [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (33) CometFilter -Input [4]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#26)) +Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) (34) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28] +Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] (35) BroadcastExchange -Input [4]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28] +Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] (36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#25] +Right keys [1]: [c_customer_sk#27] Join type: Inner Join condition: None (37) Project [codegen id : 8] -Output [7]: [ss_ticket_number#5, bought_city#22, amt#23, profit#24, c_current_addr_sk#26, c_first_name#27, c_last_name#28] -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#22, amt#23, profit#24, c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28] +Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] (38) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#29, ca_city#30] +Output [2]: [ca_address_sk#31, ca_city#32] (39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#26] -Right keys [1]: [ca_address_sk#29] +Left keys [1]: [c_current_addr_sk#28] +Right keys [1]: [ca_address_sk#31] Join type: Inner -Join condition: NOT (ca_city#30 = bought_city#22) +Join condition: NOT (ca_city#32 = bought_city#24) (40) Project [codegen id : 8] -Output [7]: [c_last_name#28, c_first_name#27, ca_city#30, bought_city#22, ss_ticket_number#5, amt#23, profit#24] -Input [9]: [ss_ticket_number#5, bought_city#22, amt#23, profit#24, c_current_addr_sk#26, c_first_name#27, c_last_name#28, ca_address_sk#29, ca_city#30] +Output [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] +Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_city#32] (41) TakeOrderedAndProject -Input [7]: [c_last_name#28, c_first_name#27, ca_city#30, bought_city#22, ss_ticket_number#5, amt#23, profit#24] -Arguments: 100, [c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, ca_city#30 ASC NULLS FIRST, bought_city#22 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#28, c_first_name#27, ca_city#30, bought_city#22, ss_ticket_number#5, amt#23, profit#24] +Input [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] +Arguments: 100, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#24 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] ===== Subqueries ===== @@ -240,18 +242,18 @@ BroadcastExchange (46) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#31, d_dow#32] +Output [3]: [d_date_sk#10, d_year#33, d_dow#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct (43) CometFilter -Input [3]: [d_date_sk#10, d_year#31, d_dow#32] -Condition : ((d_dow#32 IN (6,0) AND d_year#31 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#10, d_year#33, d_dow#34] +Condition : ((d_dow#34 IN (6,0) AND d_year#33 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) (44) CometProject -Input [3]: [d_date_sk#10, d_year#31, d_dow#32] +Input [3]: [d_date_sk#10, d_year#33, d_dow#34] Arguments: [d_date_sk#10], [d_date_sk#10] (45) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index 4b9b7960e9..f40df7e979 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -4,9 +4,9 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 RowToColumnar WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index d7dfc7147d..d12c4ab567 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -1,51 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * ColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- CometHashAggregate (22) - : : +- CometColumnarExchange (21) - : : +- RowToColumnar (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- Window (34) - : +- * ColumnarToRow (33) - : +- CometSort (32) - : +- ReusedExchange (31) - +- BroadcastExchange (44) - +- * Project (43) - +- Window (42) - +- * ColumnarToRow (41) - +- CometSort (40) - +- ReusedExchange (39) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (32) + : : +- * Filter (31) + : : +- Window (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * ColumnarToRow (27) + : : +- CometSort (26) + : : +- CometColumnarExchange (25) + : : +- RowToColumnar (24) + : : +- * HashAggregate (23) + : : +- * ColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- BroadcastExchange (42) + : +- * Project (41) + : +- Window (40) + : +- * ColumnarToRow (39) + : +- CometSort (38) + : +- CometColumnarExchange (37) + : +- RowToColumnar (36) + : +- * HashAggregate (35) + : +- * ColumnarToRow (34) + : +- ReusedExchange (33) + +- BroadcastExchange (50) + +- * Project (49) + +- Window (48) + +- * ColumnarToRow (47) + +- CometSort (46) + +- ReusedExchange (45) (unknown) Scan parquet spark_catalog.default.item @@ -91,7 +97,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 51] +(10) ReusedExchange [Reuses operator id: 57] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -146,117 +152,142 @@ Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year# Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometHashAggregate +(22) ColumnarToRow [codegen id : 5] +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] + +(23) HashAggregate [codegen id : 5] Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#17] +Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS _w0#19] + +(24) RowToColumnar +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -(23) CometColumnarExchange -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +(25) CometColumnarExchange +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometSort -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(26) CometSort +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 5] -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +(27) ColumnarToRow [codegen id : 6] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -(26) Window -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(28) Window +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(27) Filter [codegen id : 6] -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +(29) Filter [codegen id : 7] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(28) Window -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] +(30) Window +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] + +(31) Filter [codegen id : 22] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) + +(32) Project [codegen id : 22] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] + +(33) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] + +(34) ColumnarToRow [codegen id : 12] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(29) Filter [codegen id : 19] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) +(35) HashAggregate [codegen id : 12] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] +Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] +Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] -(30) Project [codegen id : 19] -Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +(36) RowToColumnar +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -(31) ReusedExchange [Reuses operator id: 23] -Output [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +(37) CometColumnarExchange +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) CometSort -Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] -Arguments: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, s_store_name#23 ASC NULLS FIRST, s_company_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(38) CometSort +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 11] -Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +(39) ColumnarToRow [codegen id : 13] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -(34) Window -Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] -Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#27], [i_category#21, i_brand#22, s_store_name#23, s_company_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(40) Window +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(35) Project [codegen id : 12] -Output [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#17 AS sum_sales#28, rn#27] -Input [8]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17, rn#27] +(41) Project [codegen id : 14] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] -(36) BroadcastExchange -Input [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#28, rn#27] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=5] +(42) BroadcastExchange +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 19] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] -Right keys [5]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, (rn#27 + 1)] +(43) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] +Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] Join type: Inner Join condition: None -(38) Project [codegen id : 19] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] -Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#28, rn#27] +(44) Project [codegen id : 22] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -(39) ReusedExchange [Reuses operator id: 23] -Output [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] +(45) ReusedExchange [Reuses operator id: 37] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -(40) CometSort -Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17], [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, s_store_name#31 ASC NULLS FIRST, s_company_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +(46) CometSort +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 17] -Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] +(47) ColumnarToRow [codegen id : 20] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -(42) Window -Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#29, i_brand#30, s_store_name#31, s_company_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +(48) Window +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(43) Project [codegen id : 18] -Output [6]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#17 AS sum_sales#36, rn#35] -Input [8]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] +(49) Project [codegen id : 21] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] -(44) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#36, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=6] +(50) BroadcastExchange +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(45) BroadcastHashJoin [codegen id : 19] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, (rn#35 - 1)] +(51) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] +Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] Join type: Inner Join condition: None -(46) Project [codegen id : 19] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#36 AS nsum#38] -Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#36, rn#35] +(52) Project [codegen id : 22] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -(47) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +(53) TakeOrderedAndProject +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (51) -+- * ColumnarToRow (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) (unknown) Scan parquet spark_catalog.default.date_dim @@ -266,15 +297,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter +(55) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(51) BroadcastExchange +(57) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index b93090ae54..3640cb9bac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (19) + WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,72 +8,82 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (6) + WholeStageCodegen (7) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (12) + WholeStageCodegen (14) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (11) + WholeStageCodegen (13) ColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 + RowToColumnar + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (18) + BroadcastExchange #8 + WholeStageCodegen (21) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (17) + WholeStageCodegen (20) ColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt index 4c4c294385..ddceb24b2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* ColumnarToRow (30) -+- CometHashAggregate (29) +* HashAggregate (30) ++- * ColumnarToRow (29) +- CometColumnarExchange (28) +- RowToColumnar (27) +- * HashAggregate (26) @@ -161,13 +161,15 @@ Input [1]: [sum#18] Input [1]: [sum#18] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometHashAggregate +(29) ColumnarToRow [codegen id : 6] +Input [1]: [sum#18] + +(30) HashAggregate [codegen id : 6] Input [1]: [sum#18] Keys: [] Functions [1]: [sum(ss_quantity#4)] - -(30) ColumnarToRow [codegen id : 6] -Input [1]: [sum(ss_quantity)#19] +Aggregate Attributes [1]: [sum(ss_quantity#4)#19] +Results [1]: [sum(ss_quantity#4)#19 AS sum(ss_quantity)#20] ===== Subqueries ===== @@ -180,18 +182,18 @@ BroadcastExchange (35) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_year#20] +Output [2]: [d_date_sk#16, d_year#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (32) CometFilter -Input [2]: [d_date_sk#16, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#16, d_year#21] +Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#16)) (33) CometProject -Input [2]: [d_date_sk#16, d_year#20] +Input [2]: [d_date_sk#16, d_year#21] Arguments: [d_date_sk#16], [d_date_sk#16] (34) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt index fee69a7cc8..bf76932af0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [sum] + HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 1327e89162..7ec19d4046 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -1,86 +1,92 @@ == Physical Plan == -* ColumnarToRow (82) -+- CometTakeOrderedAndProject (81) - +- CometHashAggregate (80) - +- CometColumnarExchange (79) - +- RowToColumnar (78) - +- * HashAggregate (77) - +- Union (76) - :- * Project (25) - : +- * Filter (24) - : +- Window (23) - : +- * Sort (22) - : +- Window (21) - : +- * ColumnarToRow (20) - : +- CometSort (19) - : +- CometColumnarExchange (18) - : +- CometHashAggregate (17) - : +- CometColumnarExchange (16) - : +- RowToColumnar (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.web_returns (5) - : +- ReusedExchange (11) - :- * Project (50) - : +- * Filter (49) - : +- Window (48) - : +- * Sort (47) - : +- Window (46) - : +- * ColumnarToRow (45) - : +- CometSort (44) - : +- CometColumnarExchange (43) - : +- CometHashAggregate (42) - : +- CometColumnarExchange (41) - : +- RowToColumnar (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * ColumnarToRow (35) - : : +- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometBroadcastExchange (29) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (26) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (36) - +- * Project (75) - +- * Filter (74) - +- Window (73) - +- * Sort (72) - +- Window (71) - +- * ColumnarToRow (70) - +- CometSort (69) - +- CometColumnarExchange (68) - +- CometHashAggregate (67) - +- CometColumnarExchange (66) - +- RowToColumnar (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) - : +- CometProject (59) - : +- CometBroadcastHashJoin (58) - : :- CometBroadcastExchange (54) - : : +- CometProject (53) - : : +- CometFilter (52) - : : +- CometScan parquet spark_catalog.default.store_sales (51) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan parquet spark_catalog.default.store_returns (55) - +- ReusedExchange (61) +* ColumnarToRow (88) ++- CometTakeOrderedAndProject (87) + +- CometHashAggregate (86) + +- CometColumnarExchange (85) + +- RowToColumnar (84) + +- * HashAggregate (83) + +- Union (82) + :- * Project (27) + : +- * Filter (26) + : +- Window (25) + : +- * Sort (24) + : +- Window (23) + : +- * ColumnarToRow (22) + : +- CometSort (21) + : +- CometColumnarExchange (20) + : +- RowToColumnar (19) + : +- * HashAggregate (18) + : +- * ColumnarToRow (17) + : +- CometColumnarExchange (16) + : +- RowToColumnar (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- ReusedExchange (11) + :- * Project (54) + : +- * Filter (53) + : +- Window (52) + : +- * Sort (51) + : +- Window (50) + : +- * ColumnarToRow (49) + : +- CometSort (48) + : +- CometColumnarExchange (47) + : +- RowToColumnar (46) + : +- * HashAggregate (45) + : +- * ColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- RowToColumnar (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * ColumnarToRow (37) + : : +- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometBroadcastExchange (31) + : : : +- CometProject (30) + : : : +- CometFilter (29) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) + : : +- CometProject (34) + : : +- CometFilter (33) + : : +- CometScan parquet spark_catalog.default.catalog_returns (32) + : +- ReusedExchange (38) + +- * Project (81) + +- * Filter (80) + +- Window (79) + +- * Sort (78) + +- Window (77) + +- * ColumnarToRow (76) + +- CometSort (75) + +- CometColumnarExchange (74) + +- RowToColumnar (73) + +- * HashAggregate (72) + +- * ColumnarToRow (71) + +- CometColumnarExchange (70) + +- RowToColumnar (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * ColumnarToRow (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometBroadcastExchange (58) + : : +- CometProject (57) + : : +- CometFilter (56) + : : +- CometScan parquet spark_catalog.default.store_sales (55) + : +- CometProject (61) + : +- CometFilter (60) + : +- CometScan parquet spark_catalog.default.store_returns (59) + +- ReusedExchange (65) (unknown) Scan parquet spark_catalog.default.web_sales @@ -130,7 +136,7 @@ Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_re (10) ColumnarToRow [codegen id : 2] Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(11) ReusedExchange [Reuses operator id: 87] +(11) ReusedExchange [Reuses operator id: 93] Output [1]: [d_date_sk#13] (12) BroadcastHashJoin [codegen id : 2] @@ -157,324 +163,348 @@ Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25 Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(17) CometHashAggregate +(17) ColumnarToRow [codegen id : 3] +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] + +(18) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] +Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] + +(19) RowToColumnar +Input [3]: [item#30, return_ratio#31, currency_ratio#32] -(18) CometColumnarExchange -Input [3]: [item#26, return_ratio#27, currency_ratio#28] +(20) CometColumnarExchange +Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) CometSort -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: [item#26, return_ratio#27, currency_ratio#28], [return_ratio#27 ASC NULLS FIRST] +(21) CometSort +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] -(20) ColumnarToRow [codegen id : 3] -Input [3]: [item#26, return_ratio#27, currency_ratio#28] +(22) ColumnarToRow [codegen id : 4] +Input [3]: [item#30, return_ratio#31, currency_ratio#32] -(21) Window -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: [rank(return_ratio#27) windowspecdefinition(return_ratio#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#29], [return_ratio#27 ASC NULLS FIRST] +(23) Window +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(22) Sort [codegen id : 4] -Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] -Arguments: [currency_ratio#28 ASC NULLS FIRST], false, 0 +(24) Sort [codegen id : 5] +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 -(23) Window -Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] -Arguments: [rank(currency_ratio#28) windowspecdefinition(currency_ratio#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#30], [currency_ratio#28 ASC NULLS FIRST] +(25) Window +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(24) Filter [codegen id : 5] -Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] -Condition : ((return_rank#29 <= 10) OR (currency_rank#30 <= 10)) +(26) Filter [codegen id : 6] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(25) Project [codegen id : 5] -Output [5]: [web AS channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] +(27) Project [codegen id : 6] +Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] +Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(27) CometFilter -Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] -Condition : (((((((isnotnull(cs_net_profit#36) AND isnotnull(cs_net_paid#35)) AND isnotnull(cs_quantity#34)) AND (cs_net_profit#36 > 1.00)) AND (cs_net_paid#35 > 0.00)) AND (cs_quantity#34 > 0)) AND isnotnull(cs_order_number#33)) AND isnotnull(cs_item_sk#32)) +(29) CometFilter +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(28) CometProject -Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] -Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37], [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +(30) CometProject +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -(29) CometBroadcastExchange -Input [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] -Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +(31) CometBroadcastExchange +Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] +Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter -Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] -Condition : (((isnotnull(cr_return_amount#42) AND (cr_return_amount#42 > 10000.00)) AND isnotnull(cr_order_number#40)) AND isnotnull(cr_item_sk#39)) +(33) CometFilter +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) -(32) CometProject -Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] -Arguments: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42], [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] +(34) CometProject +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -(33) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] -Right output [4]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] -Arguments: [cs_order_number#33, cs_item_sk#32], [cr_order_number#40, cr_item_sk#39], Inner +(35) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner -(34) CometProject -Input [9]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] -Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] +(36) CometProject +Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -(35) ColumnarToRow [codegen id : 7] -Input [6]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] +(37) ColumnarToRow [codegen id : 8] +Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -(36) ReusedExchange [Reuses operator id: 87] -Output [1]: [d_date_sk#44] +(38) ReusedExchange [Reuses operator id: 93] +Output [1]: [d_date_sk#48] -(37) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#44] +(39) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#41] +Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(38) Project [codegen id : 7] -Output [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] -Input [7]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42, d_date_sk#44] +(40) Project [codegen id : 8] +Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] -(39) HashAggregate [codegen id : 7] -Input [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] -Keys [1]: [cs_item_sk#32] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#41, 0)), partial_sum(coalesce(cs_quantity#34, 0)), partial_sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Results [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] +(41) HashAggregate [codegen id : 8] +Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Keys [1]: [cs_item_sk#36] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] +Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -(40) RowToColumnar -Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] +(42) RowToColumnar +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -(41) CometColumnarExchange -Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] -Arguments: hashpartitioning(cs_item_sk#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(43) CometColumnarExchange +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(42) CometHashAggregate -Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] -Keys [1]: [cs_item_sk#32] -Functions [4]: [sum(coalesce(cr_return_quantity#41, 0)), sum(coalesce(cs_quantity#34, 0)), sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] +(44) ColumnarToRow [codegen id : 9] +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -(43) CometColumnarExchange -Input [3]: [item#57, return_ratio#58, currency_ratio#59] +(45) HashAggregate [codegen id : 9] +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Keys [1]: [cs_item_sk#36] +Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] +Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] + +(46) RowToColumnar +Input [3]: [item#65, return_ratio#66, currency_ratio#67] + +(47) CometColumnarExchange +Input [3]: [item#65, return_ratio#66, currency_ratio#67] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) CometSort -Input [3]: [item#57, return_ratio#58, currency_ratio#59] -Arguments: [item#57, return_ratio#58, currency_ratio#59], [return_ratio#58 ASC NULLS FIRST] +(48) CometSort +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [item#65, return_ratio#66, currency_ratio#67], [return_ratio#66 ASC NULLS FIRST] -(45) ColumnarToRow [codegen id : 8] -Input [3]: [item#57, return_ratio#58, currency_ratio#59] +(49) ColumnarToRow [codegen id : 10] +Input [3]: [item#65, return_ratio#66, currency_ratio#67] -(46) Window -Input [3]: [item#57, return_ratio#58, currency_ratio#59] -Arguments: [rank(return_ratio#58) windowspecdefinition(return_ratio#58 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#60], [return_ratio#58 ASC NULLS FIRST] +(50) Window +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] -(47) Sort [codegen id : 9] -Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] -Arguments: [currency_ratio#59 ASC NULLS FIRST], false, 0 +(51) Sort [codegen id : 11] +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 -(48) Window -Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] -Arguments: [rank(currency_ratio#59) windowspecdefinition(currency_ratio#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#61], [currency_ratio#59 ASC NULLS FIRST] +(52) Window +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] -(49) Filter [codegen id : 10] -Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] -Condition : ((return_rank#60 <= 10) OR (currency_rank#61 <= 10)) +(53) Filter [codegen id : 12] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) -(50) Project [codegen id : 10] -Output [5]: [catalog AS channel#62, item#57, return_ratio#58, return_rank#60, currency_rank#61] -Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] +(54) Project [codegen id : 12] +Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] (unknown) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] +Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#68), dynamicpruningexpression(ss_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(52) CometFilter -Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] -Condition : (((((((isnotnull(ss_net_profit#67) AND isnotnull(ss_net_paid#66)) AND isnotnull(ss_quantity#65)) AND (ss_net_profit#67 > 1.00)) AND (ss_net_paid#66 > 0.00)) AND (ss_quantity#65 > 0)) AND isnotnull(ss_ticket_number#64)) AND isnotnull(ss_item_sk#63)) +(56) CometFilter +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) -(53) CometProject -Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] -Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68], [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +(57) CometProject +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -(54) CometBroadcastExchange -Input [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] -Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +(58) CometBroadcastExchange +Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] (unknown) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] +Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(56) CometFilter -Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] -Condition : (((isnotnull(sr_return_amt#73) AND (sr_return_amt#73 > 10000.00)) AND isnotnull(sr_ticket_number#71)) AND isnotnull(sr_item_sk#70)) +(60) CometFilter +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) -(57) CometProject -Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] -Arguments: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73], [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] +(61) CometProject +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -(58) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] -Right output [4]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] -Arguments: [ss_ticket_number#64, ss_item_sk#63], [sr_ticket_number#71, sr_item_sk#70], Inner +(62) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner -(59) CometProject -Input [9]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] -Arguments: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73], [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] +(63) CometProject +Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -(60) ColumnarToRow [codegen id : 12] -Input [6]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] +(64) ColumnarToRow [codegen id : 14] +Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -(61) ReusedExchange [Reuses operator id: 87] -Output [1]: [d_date_sk#75] +(65) ReusedExchange [Reuses operator id: 93] +Output [1]: [d_date_sk#83] -(62) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [ss_sold_date_sk#68] -Right keys [1]: [d_date_sk#75] +(66) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ss_sold_date_sk#76] +Right keys [1]: [d_date_sk#83] Join type: Inner Join condition: None -(63) Project [codegen id : 12] -Output [5]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] -Input [7]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73, d_date_sk#75] +(67) Project [codegen id : 14] +Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] + +(68) HashAggregate [codegen id : 14] +Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Keys [1]: [ss_item_sk#71] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] +Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] + +(69) RowToColumnar +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -(64) HashAggregate [codegen id : 12] -Input [5]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] -Keys [1]: [ss_item_sk#63] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#72, 0)), partial_sum(coalesce(ss_quantity#65, 0)), partial_sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#76, sum#77, sum#78, isEmpty#79, sum#80, isEmpty#81] -Results [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +(70) CometColumnarExchange +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(65) RowToColumnar -Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +(71) ColumnarToRow [codegen id : 15] +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -(66) CometColumnarExchange -Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Arguments: hashpartitioning(ss_item_sk#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(72) HashAggregate [codegen id : 15] +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Keys [1]: [ss_item_sk#71] +Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] +Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] -(67) CometHashAggregate -Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [1]: [ss_item_sk#63] -Functions [4]: [sum(coalesce(sr_return_quantity#72, 0)), sum(coalesce(ss_quantity#65, 0)), sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] +(73) RowToColumnar +Input [3]: [item#100, return_ratio#101, currency_ratio#102] -(68) CometColumnarExchange -Input [3]: [item#88, return_ratio#89, currency_ratio#90] +(74) CometColumnarExchange +Input [3]: [item#100, return_ratio#101, currency_ratio#102] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(69) CometSort -Input [3]: [item#88, return_ratio#89, currency_ratio#90] -Arguments: [item#88, return_ratio#89, currency_ratio#90], [return_ratio#89 ASC NULLS FIRST] +(75) CometSort +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [item#100, return_ratio#101, currency_ratio#102], [return_ratio#101 ASC NULLS FIRST] -(70) ColumnarToRow [codegen id : 13] -Input [3]: [item#88, return_ratio#89, currency_ratio#90] +(76) ColumnarToRow [codegen id : 16] +Input [3]: [item#100, return_ratio#101, currency_ratio#102] -(71) Window -Input [3]: [item#88, return_ratio#89, currency_ratio#90] -Arguments: [rank(return_ratio#89) windowspecdefinition(return_ratio#89 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#91], [return_ratio#89 ASC NULLS FIRST] +(77) Window +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] -(72) Sort [codegen id : 14] -Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] -Arguments: [currency_ratio#90 ASC NULLS FIRST], false, 0 +(78) Sort [codegen id : 17] +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 -(73) Window -Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] -Arguments: [rank(currency_ratio#90) windowspecdefinition(currency_ratio#90 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#92], [currency_ratio#90 ASC NULLS FIRST] +(79) Window +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] -(74) Filter [codegen id : 15] -Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] -Condition : ((return_rank#91 <= 10) OR (currency_rank#92 <= 10)) +(80) Filter [codegen id : 18] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] +Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) -(75) Project [codegen id : 15] -Output [5]: [store AS channel#93, item#88, return_ratio#89, return_rank#91, currency_rank#92] -Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] +(81) Project [codegen id : 18] +Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -(76) Union +(82) Union -(77) HashAggregate [codegen id : 16] -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +(83) HashAggregate [codegen id : 19] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(78) RowToColumnar -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +(84) RowToColumnar +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(79) CometColumnarExchange -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -Arguments: hashpartitioning(channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(85) CometColumnarExchange +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(80) CometHashAggregate -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +(86) CometHashAggregate +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] -(81) CometTakeOrderedAndProject -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,return_rank#29 ASC NULLS FIRST,currency_rank#30 ASC NULLS FIRST], output=[channel#31,item#26,return_ratio#27,return_rank#29,currency_rank#30]), 100, [channel#31 ASC NULLS FIRST, return_rank#29 ASC NULLS FIRST, currency_rank#30 ASC NULLS FIRST], [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +(87) CometTakeOrderedAndProject +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(82) ColumnarToRow [codegen id : 17] -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +(88) ColumnarToRow [codegen id : 20] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (87) -+- * ColumnarToRow (86) - +- CometProject (85) - +- CometFilter (84) - +- CometScan parquet spark_catalog.default.date_dim (83) +BroadcastExchange (93) ++- * ColumnarToRow (92) + +- CometProject (91) + +- CometFilter (90) + +- CometScan parquet spark_catalog.default.date_dim (89) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#94, d_moy#95] +Output [3]: [d_date_sk#13, d_year#106, d_moy#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(84) CometFilter -Input [3]: [d_date_sk#13, d_year#94, d_moy#95] -Condition : ((((isnotnull(d_year#94) AND isnotnull(d_moy#95)) AND (d_year#94 = 2001)) AND (d_moy#95 = 12)) AND isnotnull(d_date_sk#13)) +(90) CometFilter +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) -(85) CometProject -Input [3]: [d_date_sk#13, d_year#94, d_moy#95] +(91) CometProject +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Arguments: [d_date_sk#13], [d_date_sk#13] -(86) ColumnarToRow [codegen id : 1] +(92) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(87) BroadcastExchange +(93) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#68 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index 6ea9407566..42f84a6e7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -1,123 +1,135 @@ -WholeStageCodegen (17) +WholeStageCodegen (20) ColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 RowToColumnar - WholeStageCodegen (16) + WholeStageCodegen (19) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (6) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (4) + WholeStageCodegen (5) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (3) + WholeStageCodegen (4) ColumnarToRow InputAdapter CometSort [return_ratio] CometColumnarExchange #2 - CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [ws_item_sk] #3 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (10) + RowToColumnar + WholeStageCodegen (3) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk] #3 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (12) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (9) + WholeStageCodegen (11) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (8) + WholeStageCodegen (10) ColumnarToRow InputAdapter CometSort [return_ratio] CometColumnarExchange #6 - CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [cs_item_sk] #7 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #8 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (15) + RowToColumnar + WholeStageCodegen (9) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [cs_item_sk] #7 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #8 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (18) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (14) + WholeStageCodegen (17) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (13) + WholeStageCodegen (16) ColumnarToRow InputAdapter CometSort [return_ratio] CometColumnarExchange #9 - CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [ss_item_sk] #10 - RowToColumnar - WholeStageCodegen (12) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #11 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 + RowToColumnar + WholeStageCodegen (15) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk] #10 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #11 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 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 3f204c402c..57bce0eead 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,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) +TakeOrderedAndProject (78) ++- * HashAggregate (77) + +- * ColumnarToRow (76) +- CometColumnarExchange (75) +- RowToColumnar (74) +- * HashAggregate (73) @@ -430,17 +430,19 @@ Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, su Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Arguments: hashpartitioning(channel#129, id#130, spark_grouping_id#131, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometHashAggregate +(76) ColumnarToRow [codegen id : 14] +Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(77) HashAggregate [codegen id : 14] Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Keys [3]: [channel#129, id#130, spark_grouping_id#131] Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#37)#144, sum(returns#38)#145, sum(profit#39)#146] +Results [5]: [channel#129, id#130, sum(sales#37)#144 AS sales#147, sum(returns#38)#145 AS returns#148, sum(profit#39)#146 AS profit#149] -(77) CometTakeOrderedAndProject -Input [5]: [channel#129, id#130, sales#144, returns#145, profit#146] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#129 ASC NULLS FIRST,id#130 ASC NULLS FIRST], output=[channel#129,id#130,sales#144,returns#145,profit#146]), 100, [channel#129 ASC NULLS FIRST, id#130 ASC NULLS FIRST], [channel#129, id#130, sales#144, returns#145, profit#146] - -(78) ColumnarToRow [codegen id : 14] -Input [5]: [channel#129, id#130, sales#144, returns#145, profit#146] +(78) TakeOrderedAndProject +Input [5]: [channel#129, id#130, sales#147, returns#148, profit#149] +Arguments: 100, [channel#129 ASC NULLS FIRST, id#130 ASC NULLS FIRST], [channel#129, id#130, sales#147, returns#148, profit#149] ===== Subqueries ===== @@ -453,18 +455,18 @@ BroadcastExchange (83) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#147] +Output [2]: [d_date_sk#22, d_date#150] 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 (80) CometFilter -Input [2]: [d_date_sk#22, d_date#147] -Condition : (((isnotnull(d_date#147) AND (d_date#147 >= 2000-08-23)) AND (d_date#147 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#22, d_date#150] +Condition : (((isnotnull(d_date#150) AND (d_date#150 >= 2000-08-23)) AND (d_date#150 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) (81) CometProject -Input [2]: [d_date_sk#22, d_date#147] +Input [2]: [d_date_sk#22, d_date#150] Arguments: [d_date_sk#22], [d_date_sk#22] (82) ColumnarToRow [codegen id : 1] 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 daa36d5049..689f242fb1 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,8 +1,8 @@ -WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (14) + 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] + ColumnarToRow + InputAdapter CometColumnarExchange [channel,id,spark_grouping_id] #1 RowToColumnar WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt index c30a1571cb..ea1267cb3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (30) -+- CometTakeOrderedAndProject (29) - +- CometHashAggregate (28) +TakeOrderedAndProject (30) ++- * HashAggregate (29) + +- * ColumnarToRow (28) +- CometColumnarExchange (27) +- RowToColumnar (26) +- * HashAggregate (25) @@ -158,17 +158,19 @@ Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) CometHashAggregate +(28) ColumnarToRow [codegen id : 6] +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] + +(29) HashAggregate [codegen id : 6] Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] +Aggregate Attributes [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38] +Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34 AS 30 days #39, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35 AS 31 - 60 days #40, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36 AS 61 - 90 days #41, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37 AS 91 - 120 days #42, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38 AS >120 days #43] -(29) CometTakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #34, 31 - 60 days #35, 61 - 90 days #36, 91 - 120 days #37, >120 days #38] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#12 ASC NULLS FIRST,s_company_id#13 ASC NULLS FIRST,s_street_number#14 ASC NULLS FIRST,s_street_name#15 ASC NULLS FIRST,s_street_type#16 ASC NULLS FIRST,s_suite_number#17 ASC NULLS FIRST,s_city#18 ASC NULLS FIRST,s_county#19 ASC NULLS FIRST,s_state#20 ASC NULLS FIRST,s_zip#21 ASC NULLS FIRST], output=[s_store_name#12,s_company_id#13,s_street_number#14,s_street_name#15,s_street_type#16,s_suite_number#17,s_city#18,s_county#19,s_state#20,s_zip#21,30 days #34,31 - 60 days #35,61 - 90 days #36,91 - 120 days #37,>120 days #38]), 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #34, 31 - 60 days #35, 61 - 90 days #36, 91 - 120 days #37, >120 days #38] - -(30) ColumnarToRow [codegen id : 6] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #34, 31 - 60 days #35, 61 - 90 days #36, 91 - 120 days #37, >120 days #38] +(30) TakeOrderedAndProject +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] +Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] ===== Subqueries ===== @@ -181,18 +183,18 @@ BroadcastExchange (35) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#23, d_year#39, d_moy#40] +Output [3]: [d_date_sk#23, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] ReadSchema: struct (32) CometFilter -Input [3]: [d_date_sk#23, d_year#39, d_moy#40] -Condition : ((((isnotnull(d_year#39) AND isnotnull(d_moy#40)) AND (d_year#39 = 2001)) AND (d_moy#40 = 8)) AND isnotnull(d_date_sk#23)) +Input [3]: [d_date_sk#23, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 8)) AND isnotnull(d_date_sk#23)) (33) CometProject -Input [3]: [d_date_sk#23, d_year#39, d_moy#40] +Input [3]: [d_date_sk#23, d_year#44, d_moy#45] Arguments: [d_date_sk#23], [d_date_sk#23] (34) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt index 07cfd15950..0d5c346cf5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] +TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + WholeStageCodegen (6) + HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 RowToColumnar WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt index 68387ead7b..5b392331bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt @@ -1,48 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * Filter (43) - +- Window (42) - +- * ColumnarToRow (41) - +- CometSort (40) - +- CometColumnarExchange (39) - +- CometProject (38) - +- CometSortMergeJoin (37) - :- CometSort (18) - : +- CometColumnarExchange (17) - : +- RowToColumnar (16) - : +- * Project (15) - : +- Window (14) - : +- * ColumnarToRow (13) - : +- CometSort (12) - : +- CometColumnarExchange (11) - : +- CometHashAggregate (10) - : +- CometColumnarExchange (9) - : +- RowToColumnar (8) - : +- * HashAggregate (7) - : +- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- CometSort (36) - +- CometColumnarExchange (35) - +- RowToColumnar (34) - +- * Project (33) - +- Window (32) - +- * ColumnarToRow (31) - +- CometSort (30) - +- CometColumnarExchange (29) - +- CometHashAggregate (28) - +- CometColumnarExchange (27) - +- RowToColumnar (26) - +- * HashAggregate (25) - +- * Project (24) - +- * BroadcastHashJoin Inner BuildRight (23) - :- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (22) +TakeOrderedAndProject (48) ++- * Filter (47) + +- Window (46) + +- * ColumnarToRow (45) + +- CometSort (44) + +- CometColumnarExchange (43) + +- CometProject (42) + +- CometSortMergeJoin (41) + :- CometSort (20) + : +- CometColumnarExchange (19) + : +- RowToColumnar (18) + : +- * Project (17) + : +- Window (16) + : +- * ColumnarToRow (15) + : +- CometSort (14) + : +- CometColumnarExchange (13) + : +- RowToColumnar (12) + : +- * HashAggregate (11) + : +- * ColumnarToRow (10) + : +- CometColumnarExchange (9) + : +- RowToColumnar (8) + : +- * HashAggregate (7) + : +- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- ReusedExchange (4) + +- CometSort (40) + +- CometColumnarExchange (39) + +- RowToColumnar (38) + +- * Project (37) + +- Window (36) + +- * ColumnarToRow (35) + +- CometSort (34) + +- CometColumnarExchange (33) + +- RowToColumnar (32) + +- * HashAggregate (31) + +- * ColumnarToRow (30) + +- CometColumnarExchange (29) + +- RowToColumnar (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * ColumnarToRow (23) + : +- CometFilter (22) + : +- CometScan parquet spark_catalog.default.store_sales (21) + +- ReusedExchange (24) (unknown) Scan parquet spark_catalog.default.web_sales @@ -60,7 +64,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 2] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 49] +(4) ReusedExchange [Reuses operator id: 53] Output [2]: [d_date_sk#5, d_date#6] (5) BroadcastHashJoin [codegen id : 2] @@ -87,182 +91,198 @@ Input [3]: [ws_item_sk#1, d_date#6, sum#8] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) CometHashAggregate +(10) ColumnarToRow [codegen id : 3] +Input [3]: [ws_item_sk#1, d_date#6, sum#8] + +(11) HashAggregate [codegen id : 3] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] +Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS _w0#11, ws_item_sk#1] + +(12) RowToColumnar +Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -(11) CometColumnarExchange -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +(13) CometColumnarExchange +Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(12) CometSort -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(14) CometSort +Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(13) ColumnarToRow [codegen id : 3] -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +(15) ColumnarToRow [codegen id : 4] +Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -(14) Window -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(16) Window +Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Project [codegen id : 4] -Output [3]: [item_sk#9, d_date#6, cume_sales#11] -Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] +(17) Project [codegen id : 5] +Output [3]: [item_sk#10, d_date#6, cume_sales#12] +Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] -(16) RowToColumnar -Input [3]: [item_sk#9, d_date#6, cume_sales#11] +(18) RowToColumnar +Input [3]: [item_sk#10, d_date#6, cume_sales#12] -(17) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(19) CometColumnarExchange +Input [3]: [item_sk#10, d_date#6, cume_sales#12] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(20) CometSort +Input [3]: [item_sk#10, d_date#6, cume_sales#12] +Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] +Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(20) CometFilter -Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#12) +(22) CometFilter +Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +Condition : isnotnull(ss_item_sk#13) -(21) ColumnarToRow [codegen id : 6] -Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] +(23) ColumnarToRow [codegen id : 7] +Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -(22) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#16, d_date#17] +(24) ReusedExchange [Reuses operator id: 53] +Output [2]: [d_date_sk#17, d_date#18] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#14] -Right keys [1]: [d_date_sk#16] +(25) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ss_sold_date_sk#15] +Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(24) Project [codegen id : 6] -Output [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] -Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] - -(25) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_item_sk#12, d_date#17, sum#19] - -(26) RowToColumnar -Input [3]: [ss_item_sk#12, d_date#17, sum#19] +(26) Project [codegen id : 7] +Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] +Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] -(27) CometColumnarExchange -Input [3]: [ss_item_sk#12, d_date#17, sum#19] -Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(27) HashAggregate [codegen id : 7] +Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] +Keys [2]: [ss_item_sk#13, d_date#18] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] +Aggregate Attributes [1]: [sum#19] +Results [3]: [ss_item_sk#13, d_date#18, sum#20] -(28) CometHashAggregate -Input [3]: [ss_item_sk#12, d_date#17, sum#19] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] +(28) RowToColumnar +Input [3]: [ss_item_sk#13, d_date#18, sum#20] (29) CometColumnarExchange -Input [4]: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12] -Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [3]: [ss_item_sk#13, d_date#18, sum#20] +Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometSort -Input [4]: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12] -Arguments: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] +(30) ColumnarToRow [codegen id : 8] +Input [3]: [ss_item_sk#13, d_date#18, sum#20] -(31) ColumnarToRow [codegen id : 7] -Input [4]: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12] +(31) HashAggregate [codegen id : 8] +Input [3]: [ss_item_sk#13, d_date#18, sum#20] +Keys [2]: [ss_item_sk#13, d_date#18] +Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#21] +Results [4]: [ss_item_sk#13 AS item_sk#22, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#21,17,2) AS _w0#23, ss_item_sk#13] -(32) Window -Input [4]: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12] -Arguments: [sum(_w0#21) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#22], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] +(32) RowToColumnar +Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -(33) Project [codegen id : 8] -Output [3]: [item_sk#20, d_date#17, cume_sales#22] -Input [5]: [item_sk#20, d_date#17, _w0#21, ss_item_sk#12, cume_sales#22] +(33) CometColumnarExchange +Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] +Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(34) RowToColumnar -Input [3]: [item_sk#20, d_date#17, cume_sales#22] +(34) CometSort +Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] +Arguments: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] -(35) CometColumnarExchange -Input [3]: [item_sk#20, d_date#17, cume_sales#22] -Arguments: hashpartitioning(item_sk#20, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(35) ColumnarToRow [codegen id : 9] +Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -(36) CometSort -Input [3]: [item_sk#20, d_date#17, cume_sales#22] -Arguments: [item_sk#20, d_date#17, cume_sales#22], [item_sk#20 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] +(36) Window +Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] +Arguments: [sum(_w0#23) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#24], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] -(37) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#11] -Right output [3]: [item_sk#20, d_date#17, cume_sales#22] -Arguments: [item_sk#9, d_date#6], [item_sk#20, d_date#17], FullOuter +(37) Project [codegen id : 10] +Output [3]: [item_sk#22, d_date#18, cume_sales#24] +Input [5]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13, cume_sales#24] -(38) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#20, d_date#17, cume_sales#22] -Arguments: [item_sk#23, d_date#24, web_sales#25, store_sales#26], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#20 END AS item_sk#23, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#24, cume_sales#11 AS web_sales#25, cume_sales#22 AS store_sales#26] +(38) RowToColumnar +Input [3]: [item_sk#22, d_date#18, cume_sales#24] (39) CometColumnarExchange -Input [4]: [item_sk#23, d_date#24, web_sales#25, store_sales#26] -Arguments: hashpartitioning(item_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Input [3]: [item_sk#22, d_date#18, cume_sales#24] +Arguments: hashpartitioning(item_sk#22, d_date#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] (40) CometSort -Input [4]: [item_sk#23, d_date#24, web_sales#25, store_sales#26] -Arguments: [item_sk#23, d_date#24, web_sales#25, store_sales#26], [item_sk#23 ASC NULLS FIRST, d_date#24 ASC NULLS FIRST] +Input [3]: [item_sk#22, d_date#18, cume_sales#24] +Arguments: [item_sk#22, d_date#18, cume_sales#24], [item_sk#22 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] + +(41) CometSortMergeJoin +Left output [3]: [item_sk#10, d_date#6, cume_sales#12] +Right output [3]: [item_sk#22, d_date#18, cume_sales#24] +Arguments: [item_sk#10, d_date#6], [item_sk#22, d_date#18], FullOuter + +(42) CometProject +Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#22, d_date#18, cume_sales#24] +Arguments: [item_sk#25, d_date#26, web_sales#27, store_sales#28], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#22 END AS item_sk#25, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#26, cume_sales#12 AS web_sales#27, cume_sales#24 AS store_sales#28] + +(43) CometColumnarExchange +Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] +Arguments: hashpartitioning(item_sk#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(44) CometSort +Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] +Arguments: [item_sk#25, d_date#26, web_sales#27, store_sales#28], [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 9] -Input [4]: [item_sk#23, d_date#24, web_sales#25, store_sales#26] +(45) ColumnarToRow [codegen id : 11] +Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -(42) Window -Input [4]: [item_sk#23, d_date#24, web_sales#25, store_sales#26] -Arguments: [max(web_sales#25) windowspecdefinition(item_sk#23, d_date#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#27, max(store_sales#26) windowspecdefinition(item_sk#23, d_date#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#28], [item_sk#23], [d_date#24 ASC NULLS FIRST] +(46) Window +Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] +Arguments: [max(web_sales#27) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#29, max(store_sales#28) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#30], [item_sk#25], [d_date#26 ASC NULLS FIRST] -(43) Filter [codegen id : 10] -Input [6]: [item_sk#23, d_date#24, web_sales#25, store_sales#26, web_cumulative#27, store_cumulative#28] -Condition : ((isnotnull(web_cumulative#27) AND isnotnull(store_cumulative#28)) AND (web_cumulative#27 > store_cumulative#28)) +(47) Filter [codegen id : 12] +Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] +Condition : ((isnotnull(web_cumulative#29) AND isnotnull(store_cumulative#30)) AND (web_cumulative#29 > store_cumulative#30)) -(44) TakeOrderedAndProject -Input [6]: [item_sk#23, d_date#24, web_sales#25, store_sales#26, web_cumulative#27, store_cumulative#28] -Arguments: 100, [item_sk#23 ASC NULLS FIRST, d_date#24 ASC NULLS FIRST], [item_sk#23, d_date#24, web_sales#25, store_sales#26, web_cumulative#27, store_cumulative#28] +(48) TakeOrderedAndProject +Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] +Arguments: 100, [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#29] +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#29] -Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#5)) +(50) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] +Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1200)) AND (d_month_seq#31 <= 1211)) AND isnotnull(d_date_sk#5)) -(47) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#29] +(51) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(48) ColumnarToRow [codegen id : 1] +(52) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(49) BroadcastExchange +(53) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index d46e28b44c..4814b3ce60 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (10) + WholeStageCodegen (12) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (9) + WholeStageCodegen (11) ColumnarToRow InputAdapter CometSort [item_sk,d_date] @@ -13,59 +13,67 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometSort [item_sk,d_date] CometColumnarExchange [item_sk,d_date] #2 RowToColumnar - WholeStageCodegen (4) + WholeStageCodegen (5) Project [item_sk,d_date,cume_sales] InputAdapter Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (3) + WholeStageCodegen (4) ColumnarToRow InputAdapter CometSort [ws_item_sk,d_date] CometColumnarExchange [ws_item_sk] #3 - CometHashAggregate [ws_item_sk,d_date,sum] - CometColumnarExchange [ws_item_sk,d_date] #4 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk,d_date] #4 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 CometSort [item_sk,d_date] CometColumnarExchange [item_sk,d_date] #6 RowToColumnar - WholeStageCodegen (8) + WholeStageCodegen (10) Project [item_sk,d_date,cume_sales] InputAdapter Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (7) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometSort [ss_item_sk,d_date] CometColumnarExchange [ss_item_sk] #7 - CometHashAggregate [ss_item_sk,d_date,sum] - CometColumnarExchange [ss_item_sk,d_date] #8 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk,d_date] #8 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt index d6e0e7ab24..d95223d318 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometHashAggregate (21) +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- * ColumnarToRow (21) +- CometColumnarExchange (20) +- RowToColumnar (19) +- * HashAggregate (18) @@ -117,15 +117,17 @@ Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometHashAggregate +(21) ColumnarToRow [codegen id : 4] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] + +(22) HashAggregate [codegen id : 4] Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] +Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(22) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), 100, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] - -(23) ColumnarToRow [codegen id : 4] -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] +(23) TakeOrderedAndProject +Input [4]: [d_year#2, brand_id#14, brand#15, ext_price#16] +Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, ext_price#16] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt index 7607d1ccdd..8d8488b9fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,ext_price,brand_id,brand] - CometHashAggregate [d_year,i_brand,i_brand_id,sum] +TakeOrderedAndProject [d_year,ext_price,brand_id,brand] + WholeStageCodegen (4) + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [d_year,i_brand,i_brand_id] #1 RowToColumnar WholeStageCodegen (3) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 5cb48513fc..251b83b596 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -1,34 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * ColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- CometHashAggregate (23) - +- CometColumnarExchange (22) - +- RowToColumnar (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.store (14) +TakeOrderedAndProject (32) ++- * Project (31) + +- * Filter (30) + +- Window (29) + +- * ColumnarToRow (28) + +- CometSort (27) + +- CometColumnarExchange (26) + +- RowToColumnar (25) + +- * HashAggregate (24) + +- * ColumnarToRow (23) + +- CometColumnarExchange (22) + +- RowToColumnar (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.store_sales (5) + : +- ReusedExchange (11) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) (unknown) Scan parquet spark_catalog.default.item @@ -78,7 +80,7 @@ Join condition: None Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -(11) ReusedExchange [Reuses operator id: 35] +(11) ReusedExchange [Reuses operator id: 37] Output [2]: [d_date_sk#15, d_qoy#16] (12) BroadcastHashJoin [codegen id : 4] @@ -133,67 +135,75 @@ Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometHashAggregate +(23) ColumnarToRow [codegen id : 5] +Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] + +(24) HashAggregate [codegen id : 5] Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] Keys [2]: [i_manufact_id#5, d_qoy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] +Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] + +(25) RowToColumnar +Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -(24) CometColumnarExchange -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +(26) CometColumnarExchange +Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) CometSort -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] +(27) CometSort +Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] -(26) ColumnarToRow [codegen id : 5] -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +(28) ColumnarToRow [codegen id : 6] +Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -(27) Window -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] +(29) Window +Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] -(28) Filter [codegen id : 6] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END +(30) Filter [codegen id : 7] +Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] +Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 6] -Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] +(31) Project [codegen id : 7] +Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -(30) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +(32) TakeOrderedAndProject +Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#23, d_qoy#16] +Output [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#23, d_qoy#16] -Condition : (d_month_seq#23 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) +(34) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] +Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(33) CometProject -Input [3]: [d_date_sk#15, d_month_seq#23, d_qoy#16] +(35) CometProject +Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] -(34) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#16] -(35) BroadcastExchange +(37) BroadcastExchange Input [2]: [d_date_sk#15, d_qoy#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index af0269ae39..c0908b5397 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -1,51 +1,55 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (6) + WholeStageCodegen (7) Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] - WholeStageCodegen (5) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometSort [i_manufact_id] CometColumnarExchange [i_manufact_id] #1 - CometHashAggregate [i_manufact_id,d_qoy,sum] - CometColumnarExchange [i_manufact_id,d_qoy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - Project [i_manufact_id,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id,d_qoy] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + Project [i_manufact_id,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_category,i_class,i_brand,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_qoy] #4 InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_qoy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 5baa553cf1..046229f1e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -1,63 +1,65 @@ == Physical Plan == -* ColumnarToRow (59) -+- CometTakeOrderedAndProject (58) - +- CometHashAggregate (57) - +- CometColumnarExchange (56) - +- CometHashAggregate (55) - +- CometHashAggregate (54) - +- CometColumnarExchange (53) - +- RowToColumnar (52) - +- * HashAggregate (51) - +- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (47) - : +- * BroadcastHashJoin Inner BuildRight (46) - : :- * Project (41) - : : +- * BroadcastHashJoin Inner BuildRight (40) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (29) - : : : : +- CometHashAggregate (28) - : : : : +- CometColumnarExchange (27) - : : : : +- RowToColumnar (26) - : : : : +- * HashAggregate (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * ColumnarToRow (8) - : : : : : : : +- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (4) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- CometProject (11) - : : : : : : +- CometFilter (10) - : : : : : : +- CometScan parquet spark_catalog.default.item (9) - : : : : : +- ReusedExchange (16) - : : : : +- BroadcastExchange (22) - : : : : +- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (33) - : : : +- * ColumnarToRow (32) - : : : +- CometFilter (31) - : : : +- CometScan parquet spark_catalog.default.store_sales (30) - : : +- BroadcastExchange (39) - : : +- * ColumnarToRow (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.customer_address (36) - : +- BroadcastExchange (45) - : +- * ColumnarToRow (44) - : +- CometFilter (43) - : +- CometScan parquet spark_catalog.default.store (42) - +- ReusedExchange (48) +TakeOrderedAndProject (61) ++- * HashAggregate (60) + +- * ColumnarToRow (59) + +- CometColumnarExchange (58) + +- RowToColumnar (57) + +- * HashAggregate (56) + +- * HashAggregate (55) + +- * ColumnarToRow (54) + +- CometColumnarExchange (53) + +- RowToColumnar (52) + +- * HashAggregate (51) + +- * Project (50) + +- * BroadcastHashJoin Inner BuildRight (49) + :- * Project (47) + : +- * BroadcastHashJoin Inner BuildRight (46) + : :- * Project (41) + : : +- * BroadcastHashJoin Inner BuildRight (40) + : : :- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (29) + : : : : +- CometHashAggregate (28) + : : : : +- CometColumnarExchange (27) + : : : : +- RowToColumnar (26) + : : : : +- * HashAggregate (25) + : : : : +- * Project (24) + : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * ColumnarToRow (8) + : : : : : : : +- CometUnion (7) + : : : : : : : :- CometProject (3) + : : : : : : : : +- CometFilter (2) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : +- CometProject (6) + : : : : : : : +- CometFilter (5) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (4) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * ColumnarToRow (12) + : : : : : : +- CometProject (11) + : : : : : : +- CometFilter (10) + : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : +- ReusedExchange (16) + : : : : +- BroadcastExchange (22) + : : : : +- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (33) + : : : +- * ColumnarToRow (32) + : : : +- CometFilter (31) + : : : +- CometScan parquet spark_catalog.default.store_sales (30) + : : +- BroadcastExchange (39) + : : +- * ColumnarToRow (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.customer_address (36) + : +- BroadcastExchange (45) + : +- * ColumnarToRow (44) + : +- CometFilter (43) + : +- CometScan parquet spark_catalog.default.store (42) + +- ReusedExchange (48) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -131,7 +133,7 @@ Join condition: None Output [2]: [sold_date_sk#5, customer_sk#6] Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] -(16) ReusedExchange [Reuses operator id: 64] +(16) ReusedExchange [Reuses operator id: 66] Output [1]: [d_date_sk#17] (17) BroadcastHashJoin [codegen id : 4] @@ -279,7 +281,7 @@ Join condition: None Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26, s_county#27, s_state#28] -(48) ReusedExchange [Reuses operator id: 69] +(48) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#29] (49) BroadcastHashJoin [codegen id : 9] @@ -306,178 +308,190 @@ Input [2]: [c_customer_sk#18, sum#31] Input [2]: [c_customer_sk#18, sum#31] Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(54) CometHashAggregate +(54) ColumnarToRow [codegen id : 10] +Input [2]: [c_customer_sk#18, sum#31] + +(55) HashAggregate [codegen id : 10] Input [2]: [c_customer_sk#18, sum#31] Keys [1]: [c_customer_sk#18] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#32] +Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#32,17,2) / 50) as int) AS segment#33] -(55) CometHashAggregate -Input [1]: [segment#32] -Keys [1]: [segment#32] +(56) HashAggregate [codegen id : 10] +Input [1]: [segment#33] +Keys [1]: [segment#33] Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#34] +Results [2]: [segment#33, count#35] -(56) CometColumnarExchange -Input [2]: [segment#32, count#33] -Arguments: hashpartitioning(segment#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +(57) RowToColumnar +Input [2]: [segment#33, count#35] -(57) CometHashAggregate -Input [2]: [segment#32, count#33] -Keys [1]: [segment#32] -Functions [1]: [count(1)] +(58) CometColumnarExchange +Input [2]: [segment#33, count#35] +Arguments: hashpartitioning(segment#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(58) CometTakeOrderedAndProject -Input [3]: [segment#32, num_customers#34, segment_base#35] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#32 ASC NULLS FIRST,num_customers#34 ASC NULLS FIRST], output=[segment#32,num_customers#34,segment_base#35]), 100, [segment#32 ASC NULLS FIRST, num_customers#34 ASC NULLS FIRST], [segment#32, num_customers#34, segment_base#35] +(59) ColumnarToRow [codegen id : 11] +Input [2]: [segment#33, count#35] + +(60) HashAggregate [codegen id : 11] +Input [2]: [segment#33, count#35] +Keys [1]: [segment#33] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#36] +Results [3]: [segment#33, count(1)#36 AS num_customers#37, (segment#33 * 50) AS segment_base#38] -(59) ColumnarToRow [codegen id : 10] -Input [3]: [segment#32, num_customers#34, segment_base#35] +(61) TakeOrderedAndProject +Input [3]: [segment#33, num_customers#37, segment_base#38] +Arguments: 100, [segment#33 ASC NULLS FIRST, num_customers#37 ASC NULLS FIRST], [segment#33, num_customers#37, segment_base#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (64) -+- * ColumnarToRow (63) - +- CometProject (62) - +- CometFilter (61) - +- CometScan parquet spark_catalog.default.date_dim (60) +BroadcastExchange (66) ++- * ColumnarToRow (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#36, d_moy#37] +Output [3]: [d_date_sk#17, d_year#39, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(61) CometFilter -Input [3]: [d_date_sk#17, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_moy#37) AND isnotnull(d_year#36)) AND (d_moy#37 = 12)) AND (d_year#36 = 1998)) AND isnotnull(d_date_sk#17)) +(63) CometFilter +Input [3]: [d_date_sk#17, d_year#39, d_moy#40] +Condition : ((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 = 12)) AND (d_year#39 = 1998)) AND isnotnull(d_date_sk#17)) -(62) CometProject -Input [3]: [d_date_sk#17, d_year#36, d_moy#37] +(64) CometProject +Input [3]: [d_date_sk#17, d_year#39, d_moy#40] Arguments: [d_date_sk#17], [d_date_sk#17] -(63) ColumnarToRow [codegen id : 1] +(65) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] -(64) BroadcastExchange +(66) BroadcastExchange Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (69) -+- * ColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan parquet spark_catalog.default.date_dim (65) +BroadcastExchange (71) ++- * ColumnarToRow (70) + +- CometProject (69) + +- CometFilter (68) + +- CometScan parquet spark_catalog.default.date_dim (67) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#29, d_month_seq#38] +Output [2]: [d_date_sk#29, d_month_seq#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(66) CometFilter -Input [2]: [d_date_sk#29, d_month_seq#38] -Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#29)) +(68) CometFilter +Input [2]: [d_date_sk#29, d_month_seq#41] +Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= Subquery scalar-subquery#42, [id=#43])) AND (d_month_seq#41 <= Subquery scalar-subquery#44, [id=#45])) AND isnotnull(d_date_sk#29)) -(67) CometProject -Input [2]: [d_date_sk#29, d_month_seq#38] +(69) CometProject +Input [2]: [d_date_sk#29, d_month_seq#41] Arguments: [d_date_sk#29], [d_date_sk#29] -(68) ColumnarToRow [codegen id : 1] +(70) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#29] -(69) BroadcastExchange +(71) BroadcastExchange Input [1]: [d_date_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:4 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#39, [id=#40] -* ColumnarToRow (76) -+- CometHashAggregate (75) - +- CometColumnarExchange (74) - +- CometHashAggregate (73) - +- CometProject (72) - +- CometFilter (71) - +- CometScan parquet spark_catalog.default.date_dim (70) +Subquery:4 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#42, [id=#43] +* ColumnarToRow (78) ++- CometHashAggregate (77) + +- CometColumnarExchange (76) + +- CometHashAggregate (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#43, d_year#44, d_moy#45] +Output [3]: [d_month_seq#46, d_year#47, d_moy#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(71) CometFilter -Input [3]: [d_month_seq#43, d_year#44, d_moy#45] -Condition : (((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 1998)) AND (d_moy#45 = 12)) +(73) CometFilter +Input [3]: [d_month_seq#46, d_year#47, d_moy#48] +Condition : (((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 12)) -(72) CometProject -Input [3]: [d_month_seq#43, d_year#44, d_moy#45] -Arguments: [(d_month_seq + 1)#46], [(d_month_seq#43 + 1) AS (d_month_seq + 1)#46] +(74) CometProject +Input [3]: [d_month_seq#46, d_year#47, d_moy#48] +Arguments: [(d_month_seq + 1)#49], [(d_month_seq#46 + 1) AS (d_month_seq + 1)#49] -(73) CometHashAggregate -Input [1]: [(d_month_seq + 1)#46] -Keys [1]: [(d_month_seq + 1)#46] +(75) CometHashAggregate +Input [1]: [(d_month_seq + 1)#49] +Keys [1]: [(d_month_seq + 1)#49] Functions: [] -(74) CometColumnarExchange -Input [1]: [(d_month_seq + 1)#46] -Arguments: hashpartitioning((d_month_seq + 1)#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(76) CometColumnarExchange +Input [1]: [(d_month_seq + 1)#49] +Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(75) CometHashAggregate -Input [1]: [(d_month_seq + 1)#46] -Keys [1]: [(d_month_seq + 1)#46] +(77) CometHashAggregate +Input [1]: [(d_month_seq + 1)#49] +Keys [1]: [(d_month_seq + 1)#49] Functions: [] -(76) ColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#46] +(78) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 1)#49] -Subquery:5 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#41, [id=#42] -* ColumnarToRow (83) -+- CometHashAggregate (82) - +- CometColumnarExchange (81) - +- CometHashAggregate (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan parquet spark_catalog.default.date_dim (77) +Subquery:5 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* ColumnarToRow (85) ++- CometHashAggregate (84) + +- CometColumnarExchange (83) + +- CometHashAggregate (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan parquet spark_catalog.default.date_dim (79) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#47, d_year#48, d_moy#49] +Output [3]: [d_month_seq#50, d_year#51, d_moy#52] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(78) CometFilter -Input [3]: [d_month_seq#47, d_year#48, d_moy#49] -Condition : (((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 1998)) AND (d_moy#49 = 12)) +(80) CometFilter +Input [3]: [d_month_seq#50, d_year#51, d_moy#52] +Condition : (((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 1998)) AND (d_moy#52 = 12)) -(79) CometProject -Input [3]: [d_month_seq#47, d_year#48, d_moy#49] -Arguments: [(d_month_seq + 3)#50], [(d_month_seq#47 + 3) AS (d_month_seq + 3)#50] +(81) CometProject +Input [3]: [d_month_seq#50, d_year#51, d_moy#52] +Arguments: [(d_month_seq + 3)#53], [(d_month_seq#50 + 3) AS (d_month_seq + 3)#53] -(80) CometHashAggregate -Input [1]: [(d_month_seq + 3)#50] -Keys [1]: [(d_month_seq + 3)#50] +(82) CometHashAggregate +Input [1]: [(d_month_seq + 3)#53] +Keys [1]: [(d_month_seq + 3)#53] Functions: [] -(81) CometColumnarExchange -Input [1]: [(d_month_seq + 3)#50] -Arguments: hashpartitioning((d_month_seq + 3)#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +(83) CometColumnarExchange +Input [1]: [(d_month_seq + 3)#53] +Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(82) CometHashAggregate -Input [1]: [(d_month_seq + 3)#50] -Keys [1]: [(d_month_seq + 3)#50] +(84) CometHashAggregate +Input [1]: [(d_month_seq + 3)#53] +Keys [1]: [(d_month_seq + 3)#53] Functions: [] -(83) ColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#50] +(85) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 3)#53] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index 0d8b894b24..43ff7aad8f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -1,119 +1,123 @@ -WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [segment,num_customers,segment_base] - CometHashAggregate [segment,count] +TakeOrderedAndProject [segment,num_customers,segment_base] + WholeStageCodegen (11) + HashAggregate [segment,count] [count(1),num_customers,segment_base,count] + ColumnarToRow + InputAdapter CometColumnarExchange [segment] #1 - CometHashAggregate [segment] - CometHashAggregate [c_customer_sk,sum] - CometColumnarExchange [c_customer_sk] #2 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] - Project [c_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - BroadcastHashJoin [customer_sk,c_customer_sk] - Project [customer_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometUnion - CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [segment] [count,count] + HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_sk] #2 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] + Project [c_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ca_county,ca_state,s_county,s_state] + Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [c_customer_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk] + BroadcastHashJoin [customer_sk,c_customer_sk] + Project [customer_sk] + BroadcastHashJoin [sold_date_sk,d_date_sk] + Project [sold_date_sk,customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_category,i_class,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) + BroadcastExchange #6 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + Subquery #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 1)] + CometColumnarExchange [(d_month_seq + 1)] #9 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 3)] + CometColumnarExchange [(d_month_seq + 3)] #10 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_county,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) + BroadcastExchange #12 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - Subquery #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometColumnarExchange [(d_month_seq + 1)] #9 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometColumnarExchange [(d_month_seq + 3)] #10 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometFilter [s_county,s_state] + CometScan parquet spark_catalog.default.store [s_county,s_state] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_county,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [s_county,s_state] - CometScan parquet spark_catalog.default.store [s_county,s_state] - InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [d_date_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt index 9dc4ba9896..402566a451 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometHashAggregate (21) +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- * ColumnarToRow (21) +- CometColumnarExchange (20) +- RowToColumnar (19) +- * HashAggregate (18) @@ -117,15 +117,17 @@ Input [3]: [i_brand#9, i_brand_id#8, sum#12] Input [3]: [i_brand#9, i_brand_id#8, sum#12] Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometHashAggregate +(21) ColumnarToRow [codegen id : 4] +Input [3]: [i_brand#9, i_brand_id#8, sum#12] + +(22) HashAggregate [codegen id : 4] Input [3]: [i_brand#9, i_brand_id#8, sum#12] Keys [2]: [i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] +Results [3]: [i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(22) CometTakeOrderedAndProject -Input [3]: [brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), 100, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] - -(23) ColumnarToRow [codegen id : 4] -Input [3]: [brand_id#13, brand#14, ext_price#15] +(23) TakeOrderedAndProject +Input [3]: [brand_id#14, brand#15, ext_price#16] +Arguments: 100, [ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [brand_id#14, brand#15, ext_price#16] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt index 368ca6f56a..30cc73bfd4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ext_price,brand_id,brand] - CometHashAggregate [i_brand,i_brand_id,sum] +TakeOrderedAndProject [ext_price,brand_id,brand] + WholeStageCodegen (4) + HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [i_brand,i_brand_id] #1 RowToColumnar WholeStageCodegen (3) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 15d5137015..64b27c6b38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -1,71 +1,75 @@ == Physical Plan == -* ColumnarToRow (67) -+- CometTakeOrderedAndProject (66) - +- CometHashAggregate (65) - +- CometColumnarExchange (64) - +- CometHashAggregate (63) - +- CometUnion (62) - :- CometHashAggregate (29) - : +- CometColumnarExchange (28) - : +- RowToColumnar (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - :- CometHashAggregate (45) - : +- CometColumnarExchange (44) - : +- RowToColumnar (43) - : +- * HashAggregate (42) - : +- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (32) - : : : : +- CometFilter (31) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- CometHashAggregate (61) - +- CometColumnarExchange (60) - +- RowToColumnar (59) - +- * HashAggregate (58) - +- * Project (57) - +- * BroadcastHashJoin Inner BuildRight (56) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * ColumnarToRow (48) - : : : +- CometFilter (47) - : : : +- CometScan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (49) - : +- ReusedExchange (52) - +- ReusedExchange (55) +TakeOrderedAndProject (71) ++- * HashAggregate (70) + +- * ColumnarToRow (69) + +- CometColumnarExchange (68) + +- RowToColumnar (67) + +- * HashAggregate (66) + +- Union (65) + :- * HashAggregate (30) + : +- * ColumnarToRow (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- * HashAggregate (47) + : +- * ColumnarToRow (46) + : +- CometColumnarExchange (45) + : +- RowToColumnar (44) + : +- * HashAggregate (43) + : +- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * Project (36) + : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : :- * ColumnarToRow (33) + : : : : +- CometFilter (32) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : : +- ReusedExchange (34) + : : +- ReusedExchange (37) + : +- ReusedExchange (40) + +- * HashAggregate (64) + +- * ColumnarToRow (63) + +- CometColumnarExchange (62) + +- RowToColumnar (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * ColumnarToRow (50) + : : : +- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (unknown) Scan parquet spark_catalog.default.store_sales @@ -83,7 +87,7 @@ Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 72] +(4) ReusedExchange [Reuses operator id: 76] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -198,217 +202,236 @@ Input [2]: [i_item_id#10, sum#14] Input [2]: [i_item_id#10, sum#14] Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometHashAggregate +(29) ColumnarToRow [codegen id : 6] +Input [2]: [i_item_id#10, sum#14] + +(30) HashAggregate [codegen id : 6] Input [2]: [i_item_id#10, sum#14] Keys [1]: [i_item_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] +Results [2]: [i_item_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] +Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -Condition : (isnotnull(cs_bill_addr_sk#15) AND isnotnull(cs_item_sk#16)) +(32) CometFilter +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(32) ColumnarToRow [codegen id : 10] -Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] +(33) ColumnarToRow [codegen id : 11] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -(33) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#20] +(34) ReusedExchange [Reuses operator id: 76] +Output [1]: [d_date_sk#22] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] +(35) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(35) Project [codegen id : 10] -Output [3]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17] -Input [5]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#20] +(36) Project [codegen id : 11] +Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] +Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -(36) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#21] +(37) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#23] -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_addr_sk#15] -Right keys [1]: [ca_address_sk#21] +(38) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_bill_addr_sk#17] +Right keys [1]: [ca_address_sk#23] Join type: Inner Join condition: None -(38) Project [codegen id : 10] -Output [2]: [cs_item_sk#16, cs_ext_sales_price#17] -Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, ca_address_sk#21] +(39) Project [codegen id : 11] +Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] -(39) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#22, i_item_id#23] +(40) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#24, i_item_id#25] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#16] -Right keys [1]: [i_item_sk#22] +(41) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#24] Join type: Inner Join condition: None -(41) Project [codegen id : 10] -Output [2]: [cs_ext_sales_price#17, i_item_id#23] -Input [4]: [cs_item_sk#16, cs_ext_sales_price#17, i_item_sk#22, i_item_id#23] +(42) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#19, i_item_id#25] +Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] -(42) HashAggregate [codegen id : 10] -Input [2]: [cs_ext_sales_price#17, i_item_id#23] -Keys [1]: [i_item_id#23] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#17))] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_item_id#23, sum#25] +(43) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#19, i_item_id#25] +Keys [1]: [i_item_id#25] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum#26] +Results [2]: [i_item_id#25, sum#27] -(43) RowToColumnar -Input [2]: [i_item_id#23, sum#25] +(44) RowToColumnar +Input [2]: [i_item_id#25, sum#27] -(44) CometColumnarExchange -Input [2]: [i_item_id#23, sum#25] -Arguments: hashpartitioning(i_item_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(45) CometColumnarExchange +Input [2]: [i_item_id#25, sum#27] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometHashAggregate -Input [2]: [i_item_id#23, sum#25] -Keys [1]: [i_item_id#23] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#17))] +(46) ColumnarToRow [codegen id : 12] +Input [2]: [i_item_id#25, sum#27] + +(47) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#25, sum#27] +Keys [1]: [i_item_id#25] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] +Results [2]: [i_item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(47) CometFilter -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) +(49) CometFilter +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) -(48) ColumnarToRow [codegen id : 15] -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +(50) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -(49) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#31] +(51) ReusedExchange [Reuses operator id: 76] +Output [1]: [d_date_sk#35] -(50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#31] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#35] Join type: Inner Join condition: None -(51) Project [codegen id : 15] -Output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] -(52) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#32] +(54) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#36] -(53) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_bill_addr_sk#27] -Right keys [1]: [ca_address_sk#32] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#31] +Right keys [1]: [ca_address_sk#36] Join type: Inner Join condition: None -(54) Project [codegen id : 15] -Output [2]: [ws_item_sk#26, ws_ext_sales_price#28] -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -(55) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#33, i_item_id#34] +(57) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#37, i_item_id#38] -(56) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#33] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#30] +Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(57) Project [codegen id : 15] -Output [2]: [ws_ext_sales_price#28, i_item_id#34] -Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_item_id#34] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#32, i_item_id#38] +Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] + +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#32, i_item_id#38] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum#39] +Results [2]: [i_item_id#38, sum#40] -(58) HashAggregate [codegen id : 15] -Input [2]: [ws_ext_sales_price#28, i_item_id#34] -Keys [1]: [i_item_id#34] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] -Aggregate Attributes [1]: [sum#35] -Results [2]: [i_item_id#34, sum#36] +(61) RowToColumnar +Input [2]: [i_item_id#38, sum#40] -(59) RowToColumnar -Input [2]: [i_item_id#34, sum#36] +(62) CometColumnarExchange +Input [2]: [i_item_id#38, sum#40] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(60) CometColumnarExchange -Input [2]: [i_item_id#34, sum#36] -Arguments: hashpartitioning(i_item_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(63) ColumnarToRow [codegen id : 18] +Input [2]: [i_item_id#38, sum#40] -(61) CometHashAggregate -Input [2]: [i_item_id#34, sum#36] -Keys [1]: [i_item_id#34] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] +(64) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#38, sum#40] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] +Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] -(62) CometUnion -Child 0 Input [2]: [i_item_id#10, total_sales#37] -Child 1 Input [2]: [i_item_id#23, total_sales#38] -Child 2 Input [2]: [i_item_id#34, total_sales#39] +(65) Union -(63) CometHashAggregate -Input [2]: [i_item_id#10, total_sales#37] +(66) HashAggregate [codegen id : 19] +Input [2]: [i_item_id#10, total_sales#16] Keys [1]: [i_item_id#10] -Functions [1]: [partial_sum(total_sales#37)] +Functions [1]: [partial_sum(total_sales#16)] +Aggregate Attributes [2]: [sum#43, isEmpty#44] +Results [3]: [i_item_id#10, sum#45, isEmpty#46] -(64) CometColumnarExchange -Input [3]: [i_item_id#10, sum#40, isEmpty#41] +(67) RowToColumnar +Input [3]: [i_item_id#10, sum#45, isEmpty#46] + +(68) CometColumnarExchange +Input [3]: [i_item_id#10, sum#45, isEmpty#46] Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(65) CometHashAggregate -Input [3]: [i_item_id#10, sum#40, isEmpty#41] -Keys [1]: [i_item_id#10] -Functions [1]: [sum(total_sales#37)] +(69) ColumnarToRow [codegen id : 20] +Input [3]: [i_item_id#10, sum#45, isEmpty#46] -(66) CometTakeOrderedAndProject -Input [2]: [i_item_id#10, total_sales#42] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#42 ASC NULLS FIRST], output=[i_item_id#10,total_sales#42]), 100, [total_sales#42 ASC NULLS FIRST], [i_item_id#10, total_sales#42] +(70) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#10, sum#45, isEmpty#46] +Keys [1]: [i_item_id#10] +Functions [1]: [sum(total_sales#16)] +Aggregate Attributes [1]: [sum(total_sales#16)#47] +Results [2]: [i_item_id#10, sum(total_sales#16)#47 AS total_sales#48] -(67) ColumnarToRow [codegen id : 16] -Input [2]: [i_item_id#10, total_sales#42] +(71) TakeOrderedAndProject +Input [2]: [i_item_id#10, total_sales#48] +Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#10, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#43, d_moy#44] +Output [3]: [d_date_sk#6, d_year#49, d_moy#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter -Input [3]: [d_date_sk#6, d_year#43, d_moy#44] -Condition : ((((isnotnull(d_year#43) AND isnotnull(d_moy#44)) AND (d_year#43 = 2001)) AND (d_moy#44 = 2)) AND isnotnull(d_date_sk#6)) +(73) CometFilter +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2001)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#6)) -(70) CometProject -Input [3]: [d_date_sk#6, d_year#43, d_moy#44] +(74) CometProject +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] Arguments: [d_date_sk#6], [d_date_sk#6] -(71) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(72) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 68bd217820..6f7092ce81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -1,101 +1,113 @@ -WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sales,i_item_id] - CometHashAggregate [i_item_id,sum,isEmpty] +TakeOrderedAndProject [total_sales,i_item_id] + WholeStageCodegen (20) + HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #1 - CometHashAggregate [i_item_id,total_sales] - CometUnion - CometHashAggregate [i_item_id,sum] - CometColumnarExchange [i_item_id] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) + RowToColumnar + WholeStageCodegen (19) + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [i_item_id] + CometFilter [i_color] + CometScan parquet spark_catalog.default.item [i_item_id,i_color] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #7 + RowToColumnar + WholeStageCodegen (11) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow + ReusedExchange [i_item_sk,i_item_id] #5 + WholeStageCodegen (18) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #8 + RowToColumnar + WholeStageCodegen (17) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [i_item_id] - CometFilter [i_color] - CometScan parquet spark_catalog.default.item [i_item_id,i_color] - CometHashAggregate [i_item_id,sum] - CometColumnarExchange [i_item_id] #7 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - CometHashAggregate [i_item_id,sum] - CometColumnarExchange [i_item_id] #8 - RowToColumnar - WholeStageCodegen (15) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index 70e5ea9d45..87a2c73e68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -1,51 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * ColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- CometHashAggregate (22) - : : +- CometColumnarExchange (21) - : : +- RowToColumnar (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- Window (34) - : +- * ColumnarToRow (33) - : +- CometSort (32) - : +- ReusedExchange (31) - +- BroadcastExchange (44) - +- * Project (43) - +- Window (42) - +- * ColumnarToRow (41) - +- CometSort (40) - +- ReusedExchange (39) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (32) + : : +- * Filter (31) + : : +- Window (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * ColumnarToRow (27) + : : +- CometSort (26) + : : +- CometColumnarExchange (25) + : : +- RowToColumnar (24) + : : +- * HashAggregate (23) + : : +- * ColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.call_center (13) + : +- BroadcastExchange (42) + : +- * Project (41) + : +- Window (40) + : +- * ColumnarToRow (39) + : +- CometSort (38) + : +- CometColumnarExchange (37) + : +- RowToColumnar (36) + : +- * HashAggregate (35) + : +- * ColumnarToRow (34) + : +- ReusedExchange (33) + +- BroadcastExchange (50) + +- * Project (49) + +- Window (48) + +- * ColumnarToRow (47) + +- CometSort (46) + +- ReusedExchange (45) (unknown) Scan parquet spark_catalog.default.item @@ -91,7 +97,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 51] +(10) ReusedExchange [Reuses operator id: 57] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -146,117 +152,142 @@ Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometHashAggregate +(22) ColumnarToRow [codegen id : 5] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] + +(23) HashAggregate [codegen id : 5] Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#16] +Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS _w0#18] + +(24) RowToColumnar +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -(23) CometColumnarExchange -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +(25) CometColumnarExchange +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometSort -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] -Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(26) CometSort +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 5] -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +(27) ColumnarToRow [codegen id : 6] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -(26) Window -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#18], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(28) Window +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(27) Filter [codegen id : 6] -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] +(29) Filter [codegen id : 7] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(28) Window -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] -Arguments: [avg(_w0#17) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#19], [i_category#3, i_brand#2, cc_name#13, d_year#10] +(30) Window +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] + +(31) Filter [codegen id : 22] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) + +(32) Project [codegen id : 22] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] + +(33) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] + +(34) ColumnarToRow [codegen id : 12] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(29) Filter [codegen id : 19] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] -Condition : ((isnotnull(avg_monthly_sales#19) AND (avg_monthly_sales#19 > 0.000000)) AND CASE WHEN (avg_monthly_sales#19 > 0.000000) THEN ((abs((sum_sales#16 - avg_monthly_sales#19)) / avg_monthly_sales#19) > 0.1000000000000000) END) +(35) HashAggregate [codegen id : 12] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] +Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] +Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] -(30) Project [codegen id : 19] -Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] +(36) RowToColumnar +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -(31) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +(37) CometColumnarExchange +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) CometSort -Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] -Arguments: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16], [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, cc_name#22 ASC NULLS FIRST, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] +(38) CometSort +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 11] -Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +(39) ColumnarToRow [codegen id : 13] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -(34) Window -Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] -Arguments: [rank(d_year#23, d_moy#24) windowspecdefinition(i_category#20, i_brand#21, cc_name#22, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#25], [i_category#20, i_brand#21, cc_name#22], [d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] +(40) Window +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(35) Project [codegen id : 12] -Output [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#16 AS sum_sales#26, rn#25] -Input [7]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16, rn#25] +(41) Project [codegen id : 14] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] -(36) BroadcastExchange -Input [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#26, rn#25] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=5] +(42) BroadcastExchange +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 19] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] -Right keys [4]: [i_category#20, i_brand#21, cc_name#22, (rn#25 + 1)] +(43) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] +Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(38) Project [codegen id : 19] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#26] -Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, cc_name#22, sum_sales#26, rn#25] +(44) Project [codegen id : 22] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -(39) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] +(45) ReusedExchange [Reuses operator id: 37] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -(40) CometSort -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] -Arguments: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16], [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] +(46) CometSort +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 17] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] +(47) ColumnarToRow [codegen id : 20] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -(42) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] -Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] +(48) Window +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(43) Project [codegen id : 18] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#16 AS sum_sales#33, rn#32] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16, rn#32] +(49) Project [codegen id : 21] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] -(44) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#33, rn#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=6] +(50) BroadcastExchange +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(45) BroadcastHashJoin [codegen id : 19] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#32 - 1)] +(51) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] +Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(46) Project [codegen id : 19] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#26 AS psum#34, sum_sales#33 AS nsum#35] -Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#26, i_category#27, i_brand#28, cc_name#29, sum_sales#33, rn#32] +(52) Project [codegen id : 22] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -(47) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#34, nsum#35] -Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#34, nsum#35] +(53) TakeOrderedAndProject +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (51) -+- * ColumnarToRow (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) (unknown) Scan parquet spark_catalog.default.date_dim @@ -266,15 +297,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter +(55) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(51) BroadcastExchange +(57) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index b9bd07df45..a68dc18ad2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (19) + WholeStageCodegen (22) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,72 +8,82 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (6) + WholeStageCodegen (7) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy] CometColumnarExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (12) + WholeStageCodegen (14) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (11) + WholeStageCodegen (13) ColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 + CometColumnarExchange [i_category,i_brand,cc_name] #7 + RowToColumnar + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (18) + BroadcastExchange #8 + WholeStageCodegen (21) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (17) + WholeStageCodegen (20) ColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index d31d811d90..49924f25d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -4,9 +4,9 @@ TakeOrderedAndProject (55) +- * BroadcastHashJoin Inner BuildRight (53) :- * Project (36) : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (18) - : : +- CometFilter (17) - : : +- CometHashAggregate (16) + : :- * Filter (18) + : : +- * HashAggregate (17) + : : +- * ColumnarToRow (16) : : +- CometColumnarExchange (15) : : +- RowToColumnar (14) : : +- * HashAggregate (13) @@ -23,9 +23,9 @@ TakeOrderedAndProject (55) : : : +- CometScan parquet spark_catalog.default.item (4) : : +- ReusedExchange (10) : +- BroadcastExchange (34) - : +- * ColumnarToRow (33) - : +- CometFilter (32) - : +- CometHashAggregate (31) + : +- * Filter (33) + : +- * HashAggregate (32) + : +- * ColumnarToRow (31) : +- CometColumnarExchange (30) : +- RowToColumnar (29) : +- * HashAggregate (28) @@ -39,9 +39,9 @@ TakeOrderedAndProject (55) : : +- ReusedExchange (22) : +- ReusedExchange (25) +- BroadcastExchange (52) - +- * ColumnarToRow (51) - +- CometFilter (50) - +- CometHashAggregate (49) + +- * Filter (51) + +- * HashAggregate (50) + +- * ColumnarToRow (49) +- CometColumnarExchange (48) +- RowToColumnar (47) +- * HashAggregate (46) @@ -126,183 +126,189 @@ Input [2]: [i_item_id#6, sum#9] Input [2]: [i_item_id#6, sum#9] Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 15] +Input [2]: [i_item_id#6, sum#9] + +(17) HashAggregate [codegen id : 15] Input [2]: [i_item_id#6, sum#9] Keys [1]: [i_item_id#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#10] +Results [2]: [i_item_id#6 AS item_id#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#10,17,2) AS ss_item_rev#12] -(17) CometFilter -Input [2]: [item_id#10, ss_item_rev#11] -Condition : isnotnull(ss_item_rev#11) - -(18) ColumnarToRow [codegen id : 15] -Input [2]: [item_id#10, ss_item_rev#11] +(18) Filter [codegen id : 15] +Input [2]: [item_id#11, ss_item_rev#12] +Condition : isnotnull(ss_item_rev#12) (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#12, cs_ext_sales_price#13, cs_sold_date_sk#14] +Output [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (20) CometFilter -Input [3]: [cs_item_sk#12, cs_ext_sales_price#13, cs_sold_date_sk#14] -Condition : isnotnull(cs_item_sk#12) +Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +Condition : isnotnull(cs_item_sk#13) (21) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#12, cs_ext_sales_price#13, cs_sold_date_sk#14] +Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] (22) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#16, i_item_id#17] +Output [2]: [i_item_sk#17, i_item_id#18] (23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#12] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [cs_item_sk#13] +Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None (24) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#13, cs_sold_date_sk#14, i_item_id#17] -Input [5]: [cs_item_sk#12, cs_ext_sales_price#13, cs_sold_date_sk#14, i_item_sk#16, i_item_id#17] +Output [3]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18] +Input [5]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_sk#17, i_item_id#18] (25) ReusedExchange [Reuses operator id: 66] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#19] (26) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [cs_sold_date_sk#15] +Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None (27) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#13, i_item_id#17] -Input [4]: [cs_ext_sales_price#13, cs_sold_date_sk#14, i_item_id#17, d_date_sk#18] +Output [2]: [cs_ext_sales_price#14, i_item_id#18] +Input [4]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18, d_date_sk#19] (28) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#13, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#13))] -Aggregate Attributes [1]: [sum#19] -Results [2]: [i_item_id#17, sum#20] +Input [2]: [cs_ext_sales_price#14, i_item_id#18] +Keys [1]: [i_item_id#18] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#14))] +Aggregate Attributes [1]: [sum#20] +Results [2]: [i_item_id#18, sum#21] (29) RowToColumnar -Input [2]: [i_item_id#17, sum#20] +Input [2]: [i_item_id#18, sum#21] (30) CometColumnarExchange -Input [2]: [i_item_id#17, sum#20] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Input [2]: [i_item_id#18, sum#21] +Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(31) CometHashAggregate -Input [2]: [i_item_id#17, sum#20] -Keys [1]: [i_item_id#17] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#13))] +(31) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_id#18, sum#21] -(32) CometFilter -Input [2]: [item_id#21, cs_item_rev#22] -Condition : isnotnull(cs_item_rev#22) +(32) HashAggregate [codegen id : 9] +Input [2]: [i_item_id#18, sum#21] +Keys [1]: [i_item_id#18] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#14))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#14))#22] +Results [2]: [i_item_id#18 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#14))#22,17,2) AS cs_item_rev#24] -(33) ColumnarToRow [codegen id : 9] -Input [2]: [item_id#21, cs_item_rev#22] +(33) Filter [codegen id : 9] +Input [2]: [item_id#23, cs_item_rev#24] +Condition : isnotnull(cs_item_rev#24) (34) BroadcastExchange -Input [2]: [item_id#21, cs_item_rev#22] +Input [2]: [item_id#23, cs_item_rev#24] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] (35) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#10] -Right keys [1]: [item_id#21] +Left keys [1]: [item_id#11] +Right keys [1]: [item_id#23] Join type: Inner -Join condition: ((((cast(ss_item_rev#11 as decimal(19,3)) >= (0.9 * cs_item_rev#22)) AND (cast(ss_item_rev#11 as decimal(20,3)) <= (1.1 * cs_item_rev#22))) AND (cast(cs_item_rev#22 as decimal(19,3)) >= (0.9 * ss_item_rev#11))) AND (cast(cs_item_rev#22 as decimal(20,3)) <= (1.1 * ss_item_rev#11))) +Join condition: ((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) (36) Project [codegen id : 15] -Output [3]: [item_id#10, ss_item_rev#11, cs_item_rev#22] -Input [4]: [item_id#10, ss_item_rev#11, item_id#21, cs_item_rev#22] +Output [3]: [item_id#11, ss_item_rev#12, cs_item_rev#24] +Input [4]: [item_id#11, ss_item_rev#12, item_id#23, cs_item_rev#24] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] +Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (38) CometFilter -Input [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] -Condition : isnotnull(ws_item_sk#23) +Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#25) (39) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25] +Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] (40) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#27, i_item_id#28] +Output [2]: [i_item_sk#29, i_item_id#30] (41) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#27] +Left keys [1]: [ws_item_sk#25] +Right keys [1]: [i_item_sk#29] Join type: Inner Join condition: None (42) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28] -Input [5]: [ws_item_sk#23, ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_sk#27, i_item_id#28] +Output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] +Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#29, i_item_id#30] (43) ReusedExchange [Reuses operator id: 66] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#31] (44) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#25] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None (45) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#24, i_item_id#28] -Input [4]: [ws_ext_sales_price#24, ws_sold_date_sk#25, i_item_id#28, d_date_sk#29] +Output [2]: [ws_ext_sales_price#26, i_item_id#30] +Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30, d_date_sk#31] (46) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#24, i_item_id#28] -Keys [1]: [i_item_id#28] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#24))] -Aggregate Attributes [1]: [sum#30] -Results [2]: [i_item_id#28, sum#31] +Input [2]: [ws_ext_sales_price#26, i_item_id#30] +Keys [1]: [i_item_id#30] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#30, sum#33] (47) RowToColumnar -Input [2]: [i_item_id#28, sum#31] +Input [2]: [i_item_id#30, sum#33] (48) CometColumnarExchange -Input [2]: [i_item_id#28, sum#31] -Arguments: hashpartitioning(i_item_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [2]: [i_item_id#30, sum#33] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(49) CometHashAggregate -Input [2]: [i_item_id#28, sum#31] -Keys [1]: [i_item_id#28] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#24))] +(49) ColumnarToRow [codegen id : 14] +Input [2]: [i_item_id#30, sum#33] -(50) CometFilter -Input [2]: [item_id#32, ws_item_rev#33] -Condition : isnotnull(ws_item_rev#33) +(50) HashAggregate [codegen id : 14] +Input [2]: [i_item_id#30, sum#33] +Keys [1]: [i_item_id#30] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#34] +Results [2]: [i_item_id#30 AS item_id#35, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#34,17,2) AS ws_item_rev#36] -(51) ColumnarToRow [codegen id : 14] -Input [2]: [item_id#32, ws_item_rev#33] +(51) Filter [codegen id : 14] +Input [2]: [item_id#35, ws_item_rev#36] +Condition : isnotnull(ws_item_rev#36) (52) BroadcastExchange -Input [2]: [item_id#32, ws_item_rev#33] +Input [2]: [item_id#35, ws_item_rev#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] (53) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#10] -Right keys [1]: [item_id#32] +Left keys [1]: [item_id#11] +Right keys [1]: [item_id#35] Join type: Inner -Join condition: ((((((((cast(ss_item_rev#11 as decimal(19,3)) >= (0.9 * ws_item_rev#33)) AND (cast(ss_item_rev#11 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(cs_item_rev#22 as decimal(19,3)) >= (0.9 * ws_item_rev#33))) AND (cast(cs_item_rev#22 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * ss_item_rev#11))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * ss_item_rev#11))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * cs_item_rev#22))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * cs_item_rev#22))) +Join condition: ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * ws_item_rev#36)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) (54) Project [codegen id : 15] -Output [8]: [item_id#10, ss_item_rev#11, (((ss_item_rev#11 / ((ss_item_rev#11 + cs_item_rev#22) + ws_item_rev#33)) / 3) * 100) AS ss_dev#34, cs_item_rev#22, (((cs_item_rev#22 / ((ss_item_rev#11 + cs_item_rev#22) + ws_item_rev#33)) / 3) * 100) AS cs_dev#35, ws_item_rev#33, (((ws_item_rev#33 / ((ss_item_rev#11 + cs_item_rev#22) + ws_item_rev#33)) / 3) * 100) AS ws_dev#36, (((ss_item_rev#11 + cs_item_rev#22) + ws_item_rev#33) / 3) AS average#37] -Input [5]: [item_id#10, ss_item_rev#11, cs_item_rev#22, item_id#32, ws_item_rev#33] +Output [8]: [item_id#11, ss_item_rev#12, (((ss_item_rev#12 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ss_dev#37, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS cs_dev#38, ws_item_rev#36, (((ws_item_rev#36 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ws_dev#39, (((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36) / 3) AS average#40] +Input [5]: [item_id#11, ss_item_rev#12, cs_item_rev#24, item_id#35, ws_item_rev#36] (55) TakeOrderedAndProject -Input [8]: [item_id#10, ss_item_rev#11, ss_dev#34, cs_item_rev#22, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] -Arguments: 100, [item_id#10 ASC NULLS FIRST, ss_item_rev#11 ASC NULLS FIRST], [item_id#10, ss_item_rev#11, ss_dev#34, cs_item_rev#22, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] +Input [8]: [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] +Arguments: 100, [item_id#11 ASC NULLS FIRST, ss_item_rev#12 ASC NULLS FIRST], [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] ===== Subqueries ===== @@ -321,56 +327,56 @@ BroadcastExchange (66) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_date#38] +Output [2]: [d_date_sk#7, d_date#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (57) CometFilter -Input [2]: [d_date_sk#7, d_date#38] +Input [2]: [d_date_sk#7, d_date#41] Condition : isnotnull(d_date_sk#7) (58) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#7, d_date#38] +Input [2]: [d_date_sk#7, d_date#41] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#39, d_week_seq#40] +Output [2]: [d_date#42, d_week_seq#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (60) CometFilter -Input [2]: [d_date#39, d_week_seq#40] -Condition : (isnotnull(d_week_seq#40) AND (d_week_seq#40 = Subquery scalar-subquery#41, [id=#42])) +Input [2]: [d_date#42, d_week_seq#43] +Condition : (isnotnull(d_week_seq#43) AND (d_week_seq#43 = Subquery scalar-subquery#44, [id=#45])) (61) CometProject -Input [2]: [d_date#39, d_week_seq#40] -Arguments: [d_date#39], [d_date#39] +Input [2]: [d_date#42, d_week_seq#43] +Arguments: [d_date#42], [d_date#42] (62) ColumnarToRow [codegen id : 1] -Input [1]: [d_date#39] +Input [1]: [d_date#42] (63) BroadcastExchange -Input [1]: [d_date#39] +Input [1]: [d_date#42] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=7] (64) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_date#38] -Right keys [1]: [d_date#39] +Left keys [1]: [d_date#41] +Right keys [1]: [d_date#42] Join type: LeftSemi Join condition: None (65) Project [codegen id : 2] Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#38] +Input [2]: [d_date_sk#7, d_date#41] (66) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 60 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:2 Hosting operator id = 60 Hosting Expression = Subquery scalar-subquery#44, [id=#45] * ColumnarToRow (70) +- CometProject (69) +- CometFilter (68) @@ -378,25 +384,25 @@ Subquery:2 Hosting operator id = 60 Hosting Expression = Subquery scalar-subquer (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#43, d_week_seq#44] +Output [2]: [d_date#46, d_week_seq#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct (68) CometFilter -Input [2]: [d_date#43, d_week_seq#44] -Condition : (isnotnull(d_date#43) AND (d_date#43 = 2000-01-03)) +Input [2]: [d_date#46, d_week_seq#47] +Condition : (isnotnull(d_date#46) AND (d_date#46 = 2000-01-03)) (69) CometProject -Input [2]: [d_date#43, d_week_seq#44] -Arguments: [d_week_seq#44], [d_week_seq#44] +Input [2]: [d_date#46, d_week_seq#47] +Arguments: [d_week_seq#47], [d_week_seq#47] (70) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#44] +Input [1]: [d_week_seq#47] -Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#25 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 63eb226bb0..efc0244704 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -4,10 +4,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] Project [item_id,ss_item_rev,cs_item_rev] BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - ColumnarToRow - InputAdapter - CometFilter [ss_item_rev] - CometHashAggregate [i_item_id,sum] + Filter [ss_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #1 RowToColumnar WholeStageCodegen (4) @@ -56,10 +56,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometFilter [cs_item_rev] - CometHashAggregate [i_item_id,sum] + Filter [cs_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #6 RowToColumnar WholeStageCodegen (8) @@ -80,10 +80,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev InputAdapter BroadcastExchange #7 WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometFilter [ws_item_rev] - CometHashAggregate [i_item_id,sum] + Filter [ws_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #8 RowToColumnar WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt index d2241863e3..1211840ec0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt @@ -6,8 +6,8 @@ TakeOrderedAndProject (47) : +- * BroadcastHashJoin Inner BuildRight (26) : :- * Project (20) : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (14) - : : : +- CometHashAggregate (13) + : : :- * HashAggregate (14) + : : : +- * ColumnarToRow (13) : : : +- CometColumnarExchange (12) : : : +- RowToColumnar (11) : : : +- * HashAggregate (10) @@ -34,8 +34,8 @@ TakeOrderedAndProject (47) +- * BroadcastHashJoin Inner BuildRight (42) :- * Project (36) : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (30) - : : +- CometHashAggregate (29) + : :- * HashAggregate (30) + : : +- * ColumnarToRow (29) : : +- ReusedExchange (28) : +- BroadcastExchange (34) : +- * ColumnarToRow (33) @@ -105,160 +105,164 @@ Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(13) CometHashAggregate +(13) ColumnarToRow [codegen id : 10] +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] + +(14) HashAggregate [codegen id : 10] Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] - -(14) ColumnarToRow [codegen id : 10] -Input [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27] +Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21,17,2) AS sun_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22,17,2) AS mon_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23,17,2) AS tue_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24,17,2) AS wed_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25,17,2) AS thu_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26,17,2) AS fri_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27,17,2) AS sat_sales#34] (unknown) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] +Output [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct (16) CometFilter -Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] -Condition : (isnotnull(s_store_sk#28) AND isnotnull(s_store_id#29)) +Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] +Condition : (isnotnull(s_store_sk#35) AND isnotnull(s_store_id#36)) (17) ColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] +Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] (18) BroadcastExchange -Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] +Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] (19) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#28] +Right keys [1]: [s_store_sk#35] Join type: Inner Join condition: None (20) Project [codegen id : 10] -Output [10]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#29, s_store_name#30] -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_sk#28, s_store_id#29, s_store_name#30] +Output [10]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37] +Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_sk#35, s_store_id#36, s_store_name#37] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#31, d_week_seq#32] +Output [2]: [d_month_seq#38, d_week_seq#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] ReadSchema: struct (22) CometFilter -Input [2]: [d_month_seq#31, d_week_seq#32] -Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1212)) AND (d_month_seq#31 <= 1223)) AND isnotnull(d_week_seq#32)) +Input [2]: [d_month_seq#38, d_week_seq#39] +Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= 1212)) AND (d_month_seq#38 <= 1223)) AND isnotnull(d_week_seq#39)) (23) CometProject -Input [2]: [d_month_seq#31, d_week_seq#32] -Arguments: [d_week_seq#32], [d_week_seq#32] +Input [2]: [d_month_seq#38, d_week_seq#39] +Arguments: [d_week_seq#39], [d_week_seq#39] (24) ColumnarToRow [codegen id : 4] -Input [1]: [d_week_seq#32] +Input [1]: [d_week_seq#39] (25) BroadcastExchange -Input [1]: [d_week_seq#32] +Input [1]: [d_week_seq#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] (26) BroadcastHashJoin [codegen id : 10] Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#32] +Right keys [1]: [d_week_seq#39] Join type: Inner Join condition: None (27) Project [codegen id : 10] -Output [10]: [s_store_name#30 AS s_store_name1#33, d_week_seq#5 AS d_week_seq1#34, s_store_id#29 AS s_store_id1#35, sun_sales#21 AS sun_sales1#36, mon_sales#22 AS mon_sales1#37, tue_sales#23 AS tue_sales1#38, wed_sales#24 AS wed_sales1#39, thu_sales#25 AS thu_sales1#40, fri_sales#26 AS fri_sales1#41, sat_sales#27 AS sat_sales1#42] -Input [11]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#29, s_store_name#30, d_week_seq#32] +Output [10]: [s_store_name#37 AS s_store_name1#40, d_week_seq#5 AS d_week_seq1#41, s_store_id#36 AS s_store_id1#42, sun_sales#28 AS sun_sales1#43, mon_sales#29 AS mon_sales1#44, tue_sales#30 AS tue_sales1#45, wed_sales#31 AS wed_sales1#46, thu_sales#32 AS thu_sales1#47, fri_sales#33 AS fri_sales1#48, sat_sales#34 AS sat_sales1#49] +Input [11]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37, d_week_seq#39] (28) ReusedExchange [Reuses operator id: 12] -Output [9]: [d_week_seq#5, ss_store_sk#1, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] +Output [9]: [d_week_seq#5, ss_store_sk#1, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] -(29) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] +(29) ColumnarToRow [codegen id : 9] +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] + +(30) HashAggregate [codegen id : 9] +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] - -(30) ColumnarToRow [codegen id : 9] -Input [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27] +Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21,17,2) AS sun_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22,17,2) AS mon_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23,17,2) AS tue_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24,17,2) AS wed_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25,17,2) AS thu_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26,17,2) AS fri_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27,17,2) AS sat_sales#34] (unknown) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#50, s_store_id#51] +Output [2]: [s_store_sk#57, s_store_id#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct (32) CometFilter -Input [2]: [s_store_sk#50, s_store_id#51] -Condition : (isnotnull(s_store_sk#50) AND isnotnull(s_store_id#51)) +Input [2]: [s_store_sk#57, s_store_id#58] +Condition : (isnotnull(s_store_sk#57) AND isnotnull(s_store_id#58)) (33) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#50, s_store_id#51] +Input [2]: [s_store_sk#57, s_store_id#58] (34) BroadcastExchange -Input [2]: [s_store_sk#50, s_store_id#51] +Input [2]: [s_store_sk#57, s_store_id#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#50] +Right keys [1]: [s_store_sk#57] Join type: Inner Join condition: None (36) Project [codegen id : 9] -Output [9]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#51] -Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_sk#50, s_store_id#51] +Output [9]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#58] +Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_sk#57, s_store_id#58] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#52, d_week_seq#53] +Output [2]: [d_month_seq#59, d_week_seq#60] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] ReadSchema: struct (38) CometFilter -Input [2]: [d_month_seq#52, d_week_seq#53] -Condition : (((isnotnull(d_month_seq#52) AND (d_month_seq#52 >= 1224)) AND (d_month_seq#52 <= 1235)) AND isnotnull(d_week_seq#53)) +Input [2]: [d_month_seq#59, d_week_seq#60] +Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1224)) AND (d_month_seq#59 <= 1235)) AND isnotnull(d_week_seq#60)) (39) CometProject -Input [2]: [d_month_seq#52, d_week_seq#53] -Arguments: [d_week_seq#53], [d_week_seq#53] +Input [2]: [d_month_seq#59, d_week_seq#60] +Arguments: [d_week_seq#60], [d_week_seq#60] (40) ColumnarToRow [codegen id : 8] -Input [1]: [d_week_seq#53] +Input [1]: [d_week_seq#60] (41) BroadcastExchange -Input [1]: [d_week_seq#53] +Input [1]: [d_week_seq#60] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#53] +Right keys [1]: [d_week_seq#60] Join type: Inner Join condition: None (43) Project [codegen id : 9] -Output [9]: [d_week_seq#5 AS d_week_seq2#54, s_store_id#51 AS s_store_id2#55, sun_sales#21 AS sun_sales2#56, mon_sales#22 AS mon_sales2#57, tue_sales#23 AS tue_sales2#58, wed_sales#24 AS wed_sales2#59, thu_sales#25 AS thu_sales2#60, fri_sales#26 AS fri_sales2#61, sat_sales#27 AS sat_sales2#62] -Input [10]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#51, d_week_seq#53] +Output [9]: [d_week_seq#5 AS d_week_seq2#61, s_store_id#58 AS s_store_id2#62, sun_sales#28 AS sun_sales2#63, mon_sales#29 AS mon_sales2#64, tue_sales#30 AS tue_sales2#65, wed_sales#31 AS wed_sales2#66, thu_sales#32 AS thu_sales2#67, fri_sales#33 AS fri_sales2#68, sat_sales#34 AS sat_sales2#69] +Input [10]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#58, d_week_seq#60] (44) BroadcastExchange -Input [9]: [d_week_seq2#54, s_store_id2#55, sun_sales2#56, mon_sales2#57, tue_sales2#58, wed_sales2#59, thu_sales2#60, fri_sales2#61, sat_sales2#62] +Input [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=7] (45) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [s_store_id1#35, d_week_seq1#34] -Right keys [2]: [s_store_id2#55, (d_week_seq2#54 - 52)] +Left keys [2]: [s_store_id1#42, d_week_seq1#41] +Right keys [2]: [s_store_id2#62, (d_week_seq2#61 - 52)] Join type: Inner Join condition: None (46) Project [codegen id : 10] -Output [10]: [s_store_name1#33, s_store_id1#35, d_week_seq1#34, (sun_sales1#36 / sun_sales2#56) AS (sun_sales1 / sun_sales2)#63, (mon_sales1#37 / mon_sales2#57) AS (mon_sales1 / mon_sales2)#64, (tue_sales1#38 / tue_sales2#58) AS (tue_sales1 / tue_sales2)#65, (wed_sales1#39 / wed_sales2#59) AS (wed_sales1 / wed_sales2)#66, (thu_sales1#40 / thu_sales2#60) AS (thu_sales1 / thu_sales2)#67, (fri_sales1#41 / fri_sales2#61) AS (fri_sales1 / fri_sales2)#68, (sat_sales1#42 / sat_sales2#62) AS (sat_sales1 / sat_sales2)#69] -Input [19]: [s_store_name1#33, d_week_seq1#34, s_store_id1#35, sun_sales1#36, mon_sales1#37, tue_sales1#38, wed_sales1#39, thu_sales1#40, fri_sales1#41, sat_sales1#42, d_week_seq2#54, s_store_id2#55, sun_sales2#56, mon_sales2#57, tue_sales2#58, wed_sales2#59, thu_sales2#60, fri_sales2#61, sat_sales2#62] +Output [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1#43 / sun_sales2#63) AS (sun_sales1 / sun_sales2)#70, (mon_sales1#44 / mon_sales2#64) AS (mon_sales1 / mon_sales2)#71, (tue_sales1#45 / tue_sales2#65) AS (tue_sales1 / tue_sales2)#72, (wed_sales1#46 / wed_sales2#66) AS (wed_sales1 / wed_sales2)#73, (thu_sales1#47 / thu_sales2#67) AS (thu_sales1 / thu_sales2)#74, (fri_sales1#48 / fri_sales2#68) AS (fri_sales1 / fri_sales2)#75, (sat_sales1#49 / sat_sales2#69) AS (sat_sales1 / sat_sales2)#76] +Input [19]: [s_store_name1#40, d_week_seq1#41, s_store_id1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] (47) TakeOrderedAndProject -Input [10]: [s_store_name1#33, s_store_id1#35, d_week_seq1#34, (sun_sales1 / sun_sales2)#63, (mon_sales1 / mon_sales2)#64, (tue_sales1 / tue_sales2)#65, (wed_sales1 / wed_sales2)#66, (thu_sales1 / thu_sales2)#67, (fri_sales1 / fri_sales2)#68, (sat_sales1 / sat_sales2)#69] -Arguments: 100, [s_store_name1#33 ASC NULLS FIRST, s_store_id1#35 ASC NULLS FIRST, d_week_seq1#34 ASC NULLS FIRST], [s_store_name1#33, s_store_id1#35, d_week_seq1#34, (sun_sales1 / sun_sales2)#63, (mon_sales1 / mon_sales2)#64, (tue_sales1 / tue_sales2)#65, (wed_sales1 / wed_sales2)#66, (thu_sales1 / thu_sales2)#67, (fri_sales1 / fri_sales2)#68, (sat_sales1 / sat_sales2)#69] +Input [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] +Arguments: 100, [s_store_name1#40 ASC NULLS FIRST, s_store_id1#42 ASC NULLS FIRST, d_week_seq1#41 ASC NULLS FIRST], [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index 6a17e8b98b..84c312a617 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -6,9 +6,9 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [d_week_seq,ss_store_sk] #1 RowToColumnar WholeStageCodegen (2) @@ -48,9 +48,9 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 InputAdapter BroadcastExchange #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index 6ae031c893..1e70914b3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* ColumnarToRow (41) -+- CometTakeOrderedAndProject (40) - +- CometFilter (39) - +- CometHashAggregate (38) +TakeOrderedAndProject (41) ++- * Filter (40) + +- * HashAggregate (39) + +- * ColumnarToRow (38) +- CometColumnarExchange (37) +- RowToColumnar (36) +- * HashAggregate (35) @@ -214,21 +214,23 @@ Input [2]: [ca_state#2, count#19] Input [2]: [ca_state#2, count#19] Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(38) CometHashAggregate +(38) ColumnarToRow [codegen id : 7] +Input [2]: [ca_state#2, count#19] + +(39) HashAggregate [codegen id : 7] Input [2]: [ca_state#2, count#19] Keys [1]: [ca_state#2] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#20] +Results [2]: [ca_state#2 AS state#21, count(1)#20 AS cnt#22] -(39) CometFilter -Input [2]: [state#20, cnt#21] -Condition : (cnt#21 >= 10) - -(40) CometTakeOrderedAndProject -Input [2]: [state#20, cnt#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#21 ASC NULLS FIRST], output=[state#20,cnt#21]), 100, [cnt#21 ASC NULLS FIRST], [state#20, cnt#21] +(40) Filter [codegen id : 7] +Input [2]: [state#21, cnt#22] +Condition : (cnt#22 >= 10) -(41) ColumnarToRow [codegen id : 7] -Input [2]: [state#20, cnt#21] +(41) TakeOrderedAndProject +Input [2]: [state#21, cnt#22] +Arguments: 100, [cnt#22 ASC NULLS FIRST], [state#21, cnt#22] ===== Subqueries ===== @@ -241,18 +243,18 @@ BroadcastExchange (46) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#22] +Output [2]: [d_date_sk#9, d_month_seq#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (43) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#22] -Condition : ((isnotnull(d_month_seq#22) AND (d_month_seq#22 = Subquery scalar-subquery#23, [id=#24])) AND isnotnull(d_date_sk#9)) +Input [2]: [d_date_sk#9, d_month_seq#23] +Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-subquery#24, [id=#25])) AND isnotnull(d_date_sk#9)) (44) CometProject -Input [2]: [d_date_sk#9, d_month_seq#22] +Input [2]: [d_date_sk#9, d_month_seq#23] Arguments: [d_date_sk#9], [d_date_sk#9] (45) ColumnarToRow [codegen id : 1] @@ -262,7 +264,7 @@ Input [1]: [d_date_sk#9] Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#23, [id=#24] +Subquery:2 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#24, [id=#25] * ColumnarToRow (53) +- CometHashAggregate (52) +- CometColumnarExchange (51) @@ -273,35 +275,35 @@ Subquery:2 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquer (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#25, d_year#26, d_moy#27] +Output [3]: [d_month_seq#26, d_year#27, d_moy#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (48) CometFilter -Input [3]: [d_month_seq#25, d_year#26, d_moy#27] -Condition : (((isnotnull(d_year#26) AND isnotnull(d_moy#27)) AND (d_year#26 = 2000)) AND (d_moy#27 = 1)) +Input [3]: [d_month_seq#26, d_year#27, d_moy#28] +Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) (49) CometProject -Input [3]: [d_month_seq#25, d_year#26, d_moy#27] -Arguments: [d_month_seq#25], [d_month_seq#25] +Input [3]: [d_month_seq#26, d_year#27, d_moy#28] +Arguments: [d_month_seq#26], [d_month_seq#26] (50) CometHashAggregate -Input [1]: [d_month_seq#25] -Keys [1]: [d_month_seq#25] +Input [1]: [d_month_seq#26] +Keys [1]: [d_month_seq#26] Functions: [] (51) CometColumnarExchange -Input [1]: [d_month_seq#25] -Arguments: hashpartitioning(d_month_seq#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Input [1]: [d_month_seq#26] +Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] (52) CometHashAggregate -Input [1]: [d_month_seq#25] -Keys [1]: [d_month_seq#25] +Input [1]: [d_month_seq#26] +Keys [1]: [d_month_seq#26] Functions: [] (53) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#25] +Input [1]: [d_month_seq#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index b855d42f90..fe28888f0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [cnt,state] - CometFilter [cnt] - CometHashAggregate [ca_state,count] +TakeOrderedAndProject [cnt,state] + WholeStageCodegen (7) + Filter [cnt] + HashAggregate [ca_state,count] [count(1),state,cnt,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_state] #1 RowToColumnar WholeStageCodegen (6) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index accdce467d..63dd79e72a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -1,71 +1,75 @@ == Physical Plan == -* ColumnarToRow (67) -+- CometTakeOrderedAndProject (66) - +- CometHashAggregate (65) - +- CometColumnarExchange (64) - +- CometHashAggregate (63) - +- CometUnion (62) - :- CometHashAggregate (29) - : +- CometColumnarExchange (28) - : +- RowToColumnar (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - :- CometHashAggregate (45) - : +- CometColumnarExchange (44) - : +- RowToColumnar (43) - : +- * HashAggregate (42) - : +- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (32) - : : : : +- CometFilter (31) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- CometHashAggregate (61) - +- CometColumnarExchange (60) - +- RowToColumnar (59) - +- * HashAggregate (58) - +- * Project (57) - +- * BroadcastHashJoin Inner BuildRight (56) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * ColumnarToRow (48) - : : : +- CometFilter (47) - : : : +- CometScan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (49) - : +- ReusedExchange (52) - +- ReusedExchange (55) +TakeOrderedAndProject (71) ++- * HashAggregate (70) + +- * ColumnarToRow (69) + +- CometColumnarExchange (68) + +- RowToColumnar (67) + +- * HashAggregate (66) + +- Union (65) + :- * HashAggregate (30) + : +- * ColumnarToRow (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- * HashAggregate (47) + : +- * ColumnarToRow (46) + : +- CometColumnarExchange (45) + : +- RowToColumnar (44) + : +- * HashAggregate (43) + : +- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * Project (36) + : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : :- * ColumnarToRow (33) + : : : : +- CometFilter (32) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : : +- ReusedExchange (34) + : : +- ReusedExchange (37) + : +- ReusedExchange (40) + +- * HashAggregate (64) + +- * ColumnarToRow (63) + +- CometColumnarExchange (62) + +- RowToColumnar (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * ColumnarToRow (50) + : : : +- CometFilter (49) + : : : +- CometScan parquet spark_catalog.default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (unknown) Scan parquet spark_catalog.default.store_sales @@ -83,7 +87,7 @@ Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 72] +(4) ReusedExchange [Reuses operator id: 76] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -198,217 +202,236 @@ Input [2]: [i_item_id#10, sum#14] Input [2]: [i_item_id#10, sum#14] Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometHashAggregate +(29) ColumnarToRow [codegen id : 6] +Input [2]: [i_item_id#10, sum#14] + +(30) HashAggregate [codegen id : 6] Input [2]: [i_item_id#10, sum#14] Keys [1]: [i_item_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] +Results [2]: [i_item_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] +Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -Condition : (isnotnull(cs_bill_addr_sk#15) AND isnotnull(cs_item_sk#16)) +(32) CometFilter +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(32) ColumnarToRow [codegen id : 10] -Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] +(33) ColumnarToRow [codegen id : 11] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -(33) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#20] +(34) ReusedExchange [Reuses operator id: 76] +Output [1]: [d_date_sk#22] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] +(35) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(35) Project [codegen id : 10] -Output [3]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17] -Input [5]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#20] +(36) Project [codegen id : 11] +Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] +Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -(36) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#21] +(37) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#23] -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_addr_sk#15] -Right keys [1]: [ca_address_sk#21] +(38) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_bill_addr_sk#17] +Right keys [1]: [ca_address_sk#23] Join type: Inner Join condition: None -(38) Project [codegen id : 10] -Output [2]: [cs_item_sk#16, cs_ext_sales_price#17] -Input [4]: [cs_bill_addr_sk#15, cs_item_sk#16, cs_ext_sales_price#17, ca_address_sk#21] +(39) Project [codegen id : 11] +Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] -(39) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#22, i_item_id#23] +(40) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#24, i_item_id#25] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#16] -Right keys [1]: [i_item_sk#22] +(41) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#24] Join type: Inner Join condition: None -(41) Project [codegen id : 10] -Output [2]: [cs_ext_sales_price#17, i_item_id#23] -Input [4]: [cs_item_sk#16, cs_ext_sales_price#17, i_item_sk#22, i_item_id#23] +(42) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#19, i_item_id#25] +Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] -(42) HashAggregate [codegen id : 10] -Input [2]: [cs_ext_sales_price#17, i_item_id#23] -Keys [1]: [i_item_id#23] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#17))] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_item_id#23, sum#25] +(43) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#19, i_item_id#25] +Keys [1]: [i_item_id#25] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum#26] +Results [2]: [i_item_id#25, sum#27] -(43) RowToColumnar -Input [2]: [i_item_id#23, sum#25] +(44) RowToColumnar +Input [2]: [i_item_id#25, sum#27] -(44) CometColumnarExchange -Input [2]: [i_item_id#23, sum#25] -Arguments: hashpartitioning(i_item_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(45) CometColumnarExchange +Input [2]: [i_item_id#25, sum#27] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometHashAggregate -Input [2]: [i_item_id#23, sum#25] -Keys [1]: [i_item_id#23] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#17))] +(46) ColumnarToRow [codegen id : 12] +Input [2]: [i_item_id#25, sum#27] + +(47) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#25, sum#27] +Keys [1]: [i_item_id#25] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] +Results [2]: [i_item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(47) CometFilter -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) +(49) CometFilter +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) -(48) ColumnarToRow [codegen id : 15] -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +(50) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -(49) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#31] +(51) ReusedExchange [Reuses operator id: 76] +Output [1]: [d_date_sk#35] -(50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#31] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#35] Join type: Inner Join condition: None -(51) Project [codegen id : 15] -Output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] -(52) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#32] +(54) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#36] -(53) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_bill_addr_sk#27] -Right keys [1]: [ca_address_sk#32] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#31] +Right keys [1]: [ca_address_sk#36] Join type: Inner Join condition: None -(54) Project [codegen id : 15] -Output [2]: [ws_item_sk#26, ws_ext_sales_price#28] -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -(55) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#33, i_item_id#34] +(57) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#37, i_item_id#38] -(56) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#33] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#30] +Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(57) Project [codegen id : 15] -Output [2]: [ws_ext_sales_price#28, i_item_id#34] -Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_item_id#34] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#32, i_item_id#38] +Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] + +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#32, i_item_id#38] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum#39] +Results [2]: [i_item_id#38, sum#40] -(58) HashAggregate [codegen id : 15] -Input [2]: [ws_ext_sales_price#28, i_item_id#34] -Keys [1]: [i_item_id#34] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] -Aggregate Attributes [1]: [sum#35] -Results [2]: [i_item_id#34, sum#36] +(61) RowToColumnar +Input [2]: [i_item_id#38, sum#40] -(59) RowToColumnar -Input [2]: [i_item_id#34, sum#36] +(62) CometColumnarExchange +Input [2]: [i_item_id#38, sum#40] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(60) CometColumnarExchange -Input [2]: [i_item_id#34, sum#36] -Arguments: hashpartitioning(i_item_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(63) ColumnarToRow [codegen id : 18] +Input [2]: [i_item_id#38, sum#40] -(61) CometHashAggregate -Input [2]: [i_item_id#34, sum#36] -Keys [1]: [i_item_id#34] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] +(64) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#38, sum#40] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] +Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] -(62) CometUnion -Child 0 Input [2]: [i_item_id#10, total_sales#37] -Child 1 Input [2]: [i_item_id#23, total_sales#38] -Child 2 Input [2]: [i_item_id#34, total_sales#39] +(65) Union -(63) CometHashAggregate -Input [2]: [i_item_id#10, total_sales#37] +(66) HashAggregate [codegen id : 19] +Input [2]: [i_item_id#10, total_sales#16] Keys [1]: [i_item_id#10] -Functions [1]: [partial_sum(total_sales#37)] +Functions [1]: [partial_sum(total_sales#16)] +Aggregate Attributes [2]: [sum#43, isEmpty#44] +Results [3]: [i_item_id#10, sum#45, isEmpty#46] -(64) CometColumnarExchange -Input [3]: [i_item_id#10, sum#40, isEmpty#41] +(67) RowToColumnar +Input [3]: [i_item_id#10, sum#45, isEmpty#46] + +(68) CometColumnarExchange +Input [3]: [i_item_id#10, sum#45, isEmpty#46] Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(65) CometHashAggregate -Input [3]: [i_item_id#10, sum#40, isEmpty#41] -Keys [1]: [i_item_id#10] -Functions [1]: [sum(total_sales#37)] +(69) ColumnarToRow [codegen id : 20] +Input [3]: [i_item_id#10, sum#45, isEmpty#46] -(66) CometTakeOrderedAndProject -Input [2]: [i_item_id#10, total_sales#42] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#10 ASC NULLS FIRST,total_sales#42 ASC NULLS FIRST], output=[i_item_id#10,total_sales#42]), 100, [i_item_id#10 ASC NULLS FIRST, total_sales#42 ASC NULLS FIRST], [i_item_id#10, total_sales#42] +(70) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#10, sum#45, isEmpty#46] +Keys [1]: [i_item_id#10] +Functions [1]: [sum(total_sales#16)] +Aggregate Attributes [1]: [sum(total_sales#16)#47] +Results [2]: [i_item_id#10, sum(total_sales#16)#47 AS total_sales#48] -(67) ColumnarToRow [codegen id : 16] -Input [2]: [i_item_id#10, total_sales#42] +(71) TakeOrderedAndProject +Input [2]: [i_item_id#10, total_sales#48] +Arguments: 100, [i_item_id#10 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#10, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#43, d_moy#44] +Output [3]: [d_date_sk#6, d_year#49, d_moy#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter -Input [3]: [d_date_sk#6, d_year#43, d_moy#44] -Condition : ((((isnotnull(d_year#43) AND isnotnull(d_moy#44)) AND (d_year#43 = 1998)) AND (d_moy#44 = 9)) AND isnotnull(d_date_sk#6)) +(73) CometFilter +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 9)) AND isnotnull(d_date_sk#6)) -(70) CometProject -Input [3]: [d_date_sk#6, d_year#43, d_moy#44] +(74) CometProject +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] Arguments: [d_date_sk#6], [d_date_sk#6] -(71) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(72) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index bad7cbf2b4..f2585fe939 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -1,101 +1,113 @@ -WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [i_item_id,sum,isEmpty] +TakeOrderedAndProject [i_item_id,total_sales] + WholeStageCodegen (20) + HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #1 - CometHashAggregate [i_item_id,total_sales] - CometUnion - CometHashAggregate [i_item_id,sum] - CometColumnarExchange [i_item_id] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) + RowToColumnar + WholeStageCodegen (19) + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [i_item_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_item_id,i_category] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #7 + RowToColumnar + WholeStageCodegen (11) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow + ReusedExchange [i_item_sk,i_item_id] #5 + WholeStageCodegen (18) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #8 + RowToColumnar + WholeStageCodegen (17) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [i_item_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_item_id,i_category] - CometHashAggregate [i_item_id,sum] - CometColumnarExchange [i_item_id] #7 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - CometHashAggregate [i_item_id,sum] - CometColumnarExchange [i_item_id] #8 - RowToColumnar - WholeStageCodegen (15) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index 69767b1f68..391b00f0ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == * Project (71) +- * BroadcastNestedLoopJoin Inner BuildRight (70) - :- * ColumnarToRow (45) - : +- CometHashAggregate (44) + :- * HashAggregate (45) + : +- * ColumnarToRow (44) : +- CometColumnarExchange (43) : +- RowToColumnar (42) : +- * HashAggregate (41) @@ -47,8 +47,8 @@ : +- CometFilter (35) : +- CometScan parquet spark_catalog.default.item (34) +- BroadcastExchange (69) - +- * ColumnarToRow (68) - +- CometHashAggregate (67) + +- * HashAggregate (68) + +- * ColumnarToRow (67) +- CometColumnarExchange (66) +- RowToColumnar (65) +- * HashAggregate (64) @@ -270,118 +270,122 @@ Input [1]: [sum#22] Input [1]: [sum#22] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(44) CometHashAggregate +(44) ColumnarToRow [codegen id : 15] +Input [1]: [sum#22] + +(45) HashAggregate [codegen id : 15] Input [1]: [sum#22] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(45) ColumnarToRow [codegen id : 15] -Input [1]: [promotions#23] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#23] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#23,17,2) AS promotions#24] (unknown) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#24, ss_customer_sk#25, ss_store_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (47) CometFilter -Input [5]: [ss_item_sk#24, ss_customer_sk#25, ss_store_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : ((isnotnull(ss_store_sk#26) AND isnotnull(ss_customer_sk#25)) AND isnotnull(ss_item_sk#24)) +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) (48) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#24, ss_customer_sk#25, ss_store_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] (49) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#30] +Output [1]: [s_store_sk#31] (50) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#26] -Right keys [1]: [s_store_sk#30] +Left keys [1]: [ss_store_sk#27] +Right keys [1]: [s_store_sk#31] Join type: Inner Join condition: None (51) Project [codegen id : 13] -Output [4]: [ss_item_sk#24, ss_customer_sk#25, ss_ext_sales_price#27, ss_sold_date_sk#28] -Input [6]: [ss_item_sk#24, ss_customer_sk#25, ss_store_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, s_store_sk#30] +Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] +Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] (52) ReusedExchange [Reuses operator id: 76] -Output [1]: [d_date_sk#31] +Output [1]: [d_date_sk#32] (53) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#31] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None (54) Project [codegen id : 13] -Output [3]: [ss_item_sk#24, ss_customer_sk#25, ss_ext_sales_price#27] -Input [5]: [ss_item_sk#24, ss_customer_sk#25, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#31] +Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] (55) ReusedExchange [Reuses operator id: 24] -Output [2]: [c_customer_sk#32, c_current_addr_sk#33] +Output [2]: [c_customer_sk#33, c_current_addr_sk#34] (56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#25] -Right keys [1]: [c_customer_sk#32] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#33] Join type: Inner Join condition: None (57) Project [codegen id : 13] -Output [3]: [ss_item_sk#24, ss_ext_sales_price#27, c_current_addr_sk#33] -Input [5]: [ss_item_sk#24, ss_customer_sk#25, ss_ext_sales_price#27, c_customer_sk#32, c_current_addr_sk#33] +Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] (58) ReusedExchange [Reuses operator id: 31] -Output [1]: [ca_address_sk#34] +Output [1]: [ca_address_sk#35] (59) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#34] +Left keys [1]: [c_current_addr_sk#34] +Right keys [1]: [ca_address_sk#35] Join type: Inner Join condition: None (60) Project [codegen id : 13] -Output [2]: [ss_item_sk#24, ss_ext_sales_price#27] -Input [4]: [ss_item_sk#24, ss_ext_sales_price#27, c_current_addr_sk#33, ca_address_sk#34] +Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] +Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] (61) ReusedExchange [Reuses operator id: 38] -Output [1]: [i_item_sk#35] +Output [1]: [i_item_sk#36] (62) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#24] -Right keys [1]: [i_item_sk#35] +Left keys [1]: [ss_item_sk#25] +Right keys [1]: [i_item_sk#36] Join type: Inner Join condition: None (63) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#27] -Input [3]: [ss_item_sk#24, ss_ext_sales_price#27, i_item_sk#35] +Output [1]: [ss_ext_sales_price#28] +Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] (64) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#27] +Input [1]: [ss_ext_sales_price#28] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#36] -Results [1]: [sum#37] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] +Aggregate Attributes [1]: [sum#37] +Results [1]: [sum#38] (65) RowToColumnar -Input [1]: [sum#37] +Input [1]: [sum#38] (66) CometColumnarExchange -Input [1]: [sum#37] +Input [1]: [sum#38] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(67) CometHashAggregate -Input [1]: [sum#37] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] +(67) ColumnarToRow [codegen id : 14] +Input [1]: [sum#38] -(68) ColumnarToRow [codegen id : 14] -Input [1]: [total#38] +(68) HashAggregate [codegen id : 14] +Input [1]: [sum#38] +Keys: [] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#28))#39] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#39,17,2) AS total#40] (69) BroadcastExchange -Input [1]: [total#38] +Input [1]: [total#40] Arguments: IdentityBroadcastMode, [plan_id=8] (70) BroadcastNestedLoopJoin [codegen id : 15] @@ -389,8 +393,8 @@ Join type: Inner Join condition: None (71) Project [codegen id : 15] -Output [3]: [promotions#23, total#38, ((cast(promotions#23 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] -Input [2]: [promotions#23, total#38] +Output [3]: [promotions#24, total#40, ((cast(promotions#24 as decimal(15,4)) / cast(total#40 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#41] +Input [2]: [promotions#24, total#40] ===== Subqueries ===== @@ -403,18 +407,18 @@ BroadcastExchange (76) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#40, d_moy#41] +Output [3]: [d_date_sk#14, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct (73) CometFilter -Input [3]: [d_date_sk#14, d_year#40, d_moy#41] -Condition : ((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 1998)) AND (d_moy#41 = 11)) AND isnotnull(d_date_sk#14)) +Input [3]: [d_date_sk#14, d_year#42, d_moy#43] +Condition : ((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 1998)) AND (d_moy#43 = 11)) AND isnotnull(d_date_sk#14)) (74) CometProject -Input [3]: [d_date_sk#14, d_year#40, d_moy#41] +Input [3]: [d_date_sk#14, d_year#42, d_moy#43] Arguments: [d_date_sk#14], [d_date_sk#14] (75) ColumnarToRow [codegen id : 1] @@ -424,6 +428,6 @@ Input [1]: [d_date_sk#14] Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 46 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 46 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index ddedbe1379..a0df279d3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -1,9 +1,9 @@ WholeStageCodegen (15) Project [promotions,total] BroadcastNestedLoopJoin - ColumnarToRow - InputAdapter - CometHashAggregate [sum] + HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (7) @@ -76,9 +76,9 @@ WholeStageCodegen (15) InputAdapter BroadcastExchange #8 WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometHashAggregate [sum] + HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange #9 RowToColumnar WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt index 427bce7dff..602ce66e9a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (34) -+- CometTakeOrderedAndProject (33) - +- CometHashAggregate (32) +TakeOrderedAndProject (34) ++- * HashAggregate (33) + +- * ColumnarToRow (32) +- CometColumnarExchange (31) +- RowToColumnar (30) +- * HashAggregate (29) @@ -179,15 +179,17 @@ Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum# Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, web_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) CometHashAggregate +(32) ColumnarToRow [codegen id : 6] +Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] + +(33) HashAggregate [codegen id : 6] Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [3]: [_groupingexpression#14, sm_type#9, web_name#11] Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] +Aggregate Attributes [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] +Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] -(33) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#25, sm_type#9, web_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#25 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,web_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#25,sm_type#9,web_name#11,30 days #26,31 - 60 days #27,61 - 90 days #28,91 - 120 days #29,>120 days #30]), 100, [substr(w_warehouse_name, 1, 20)#25 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#25, sm_type#9, web_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] - -(34) ColumnarToRow [codegen id : 6] -Input [8]: [substr(w_warehouse_name, 1, 20)#25, sm_type#9, web_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] +(34) TakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] +Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt index 4fe012742f..7f7bee1305 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] +TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + WholeStageCodegen (6) + HashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [_groupingexpression,sm_type,web_name] #1 RowToColumnar WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index 18ed395704..76b5cfd50f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -1,34 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * ColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- CometHashAggregate (23) - +- CometColumnarExchange (22) - +- RowToColumnar (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.store (14) +TakeOrderedAndProject (32) ++- * Project (31) + +- * Filter (30) + +- Window (29) + +- * ColumnarToRow (28) + +- CometSort (27) + +- CometColumnarExchange (26) + +- RowToColumnar (25) + +- * HashAggregate (24) + +- * ColumnarToRow (23) + +- CometColumnarExchange (22) + +- RowToColumnar (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.store_sales (5) + : +- ReusedExchange (11) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) (unknown) Scan parquet spark_catalog.default.item @@ -78,7 +80,7 @@ Join condition: None Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -(11) ReusedExchange [Reuses operator id: 35] +(11) ReusedExchange [Reuses operator id: 37] Output [2]: [d_date_sk#15, d_moy#16] (12) BroadcastHashJoin [codegen id : 4] @@ -133,67 +135,75 @@ Input [3]: [i_manager_id#5, d_moy#16, sum#19] Input [3]: [i_manager_id#5, d_moy#16, sum#19] Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometHashAggregate +(23) ColumnarToRow [codegen id : 5] +Input [3]: [i_manager_id#5, d_moy#16, sum#19] + +(24) HashAggregate [codegen id : 5] Input [3]: [i_manager_id#5, d_moy#16, sum#19] Keys [2]: [i_manager_id#5, d_moy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] +Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] + +(25) RowToColumnar +Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -(24) CometColumnarExchange -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +(26) CometColumnarExchange +Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) CometSort -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] +(27) CometSort +Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] -(26) ColumnarToRow [codegen id : 5] -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +(28) ColumnarToRow [codegen id : 6] +Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -(27) Window -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] +(29) Window +Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] -(28) Filter [codegen id : 6] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END +(30) Filter [codegen id : 7] +Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] +Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 6] -Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] +(31) Project [codegen id : 7] +Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -(30) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +(32) TakeOrderedAndProject +Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#23, d_moy#16] +Output [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#23, d_moy#16] -Condition : (d_month_seq#23 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) +(34) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] +Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(33) CometProject -Input [3]: [d_date_sk#15, d_month_seq#23, d_moy#16] +(35) CometProject +Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] -(34) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#16] -(35) BroadcastExchange +(37) BroadcastExchange Input [2]: [d_date_sk#15, d_moy#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index cd9cffbd1c..1d50a8d34f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -1,51 +1,55 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (6) + WholeStageCodegen (7) Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] - WholeStageCodegen (5) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometSort [i_manager_id] CometColumnarExchange [i_manager_id] #1 - CometHashAggregate [i_manager_id,d_moy,sum] - CometColumnarExchange [i_manager_id,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - Project [i_manager_id,ss_sales_price,d_moy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_manager_id,d_moy] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + Project [i_manager_id,ss_sales_price,d_moy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_manager_id] + CometFilter [i_category,i_class,i_brand,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #4 InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 3ef59799a9..007cf9bcd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) :- * Project (24) : +- * BroadcastHashJoin Inner BuildRight (23) : :- * Project (18) @@ -10,9 +10,9 @@ TakeOrderedAndProject (43) : : : +- CometFilter (2) : : : +- CometScan parquet spark_catalog.default.store (1) : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometHashAggregate (13) + : : +- * Filter (15) + : : +- * HashAggregate (14) + : : +- * ColumnarToRow (13) : : +- CometColumnarExchange (12) : : +- RowToColumnar (11) : : +- * HashAggregate (10) @@ -26,22 +26,24 @@ TakeOrderedAndProject (43) : +- * ColumnarToRow (21) : +- CometFilter (20) : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (40) - +- * ColumnarToRow (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometColumnarExchange (36) - +- CometHashAggregate (35) - +- CometHashAggregate (34) - +- CometColumnarExchange (33) - +- RowToColumnar (32) - +- * HashAggregate (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store_sales (25) - +- ReusedExchange (28) + +- BroadcastExchange (42) + +- * Filter (41) + +- * HashAggregate (40) + +- * ColumnarToRow (39) + +- CometColumnarExchange (38) + +- RowToColumnar (37) + +- * HashAggregate (36) + +- * HashAggregate (35) + +- * ColumnarToRow (34) + +- CometColumnarExchange (33) + +- RowToColumnar (32) + +- * HashAggregate (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.store_sales (25) + +- ReusedExchange (28) (unknown) Scan parquet spark_catalog.default.store @@ -55,7 +57,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) ColumnarToRow [codegen id : 8] +(3) ColumnarToRow [codegen id : 9] Input [2]: [s_store_sk#1, s_store_name#2] (unknown) Scan parquet spark_catalog.default.store_sales @@ -73,7 +75,7 @@ Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) (6) ColumnarToRow [codegen id : 2] Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -(7) ReusedExchange [Reuses operator id: 48] +(7) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#8] (8) BroadcastHashJoin [codegen id : 2] @@ -100,178 +102,192 @@ Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(13) CometHashAggregate +(13) ColumnarToRow [codegen id : 3] +Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] + +(14) HashAggregate [codegen id : 3] Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] +Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] -(14) CometFilter -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Condition : isnotnull(revenue#11) - -(15) ColumnarToRow [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +(15) Filter [codegen id : 3] +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] +Condition : isnotnull(revenue#12) (16) BroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 8] +(17) BroadcastHashJoin [codegen id : 9] Left keys [1]: [s_store_sk#1] Right keys [1]: [ss_store_sk#4] Join type: Inner Join condition: None -(18) Project [codegen id : 8] -Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] +(18) Project [codegen id : 9] +Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] +Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] (unknown) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Output [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (20) CometFilter -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Condition : isnotnull(i_item_sk#12) +Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +Condition : isnotnull(i_item_sk#13) (21) ColumnarToRow [codegen id : 4] -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] (22) BroadcastExchange -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 8] +(23) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#3] -Right keys [1]: [i_item_sk#12] +Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(24) Project [codegen id : 8] -Output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +(24) Project [codegen id : 9] +Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] +Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (26) CometFilter -Input [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_store_sk#18) +Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +Condition : isnotnull(ss_store_sk#19) (27) ColumnarToRow [codegen id : 6] -Input [4]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20] +Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -(28) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#22] +(28) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#23] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [ss_sold_date_sk#21] +Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None (30) Project [codegen id : 6] -Output [3]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19] -Input [5]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19, ss_sold_date_sk#20, d_date_sk#22] +Output [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] +Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] (31) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#17, ss_store_sk#18, ss_sales_price#19] -Keys [2]: [ss_store_sk#18, ss_item_sk#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#19))] -Aggregate Attributes [1]: [sum#23] -Results [3]: [ss_store_sk#18, ss_item_sk#17, sum#24] +Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] +Keys [2]: [ss_store_sk#19, ss_item_sk#18] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] +Aggregate Attributes [1]: [sum#24] +Results [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] (32) RowToColumnar -Input [3]: [ss_store_sk#18, ss_item_sk#17, sum#24] +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] (33) CometColumnarExchange -Input [3]: [ss_store_sk#18, ss_item_sk#17, sum#24] -Arguments: hashpartitioning(ss_store_sk#18, ss_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [3]: [ss_store_sk#18, ss_item_sk#17, sum#24] -Keys [2]: [ss_store_sk#18, ss_item_sk#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#19))] - -(35) CometHashAggregate -Input [2]: [ss_store_sk#18, revenue#25] -Keys [1]: [ss_store_sk#18] -Functions [1]: [partial_avg(revenue#25)] - -(36) CometColumnarExchange -Input [3]: [ss_store_sk#18, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometHashAggregate -Input [3]: [ss_store_sk#18, sum#26, count#27] -Keys [1]: [ss_store_sk#18] -Functions [1]: [avg(revenue#25)] - -(38) CometFilter -Input [2]: [ss_store_sk#18, ave#28] -Condition : isnotnull(ave#28) - -(39) ColumnarToRow [codegen id : 7] -Input [2]: [ss_store_sk#18, ave#28] - -(40) BroadcastExchange -Input [2]: [ss_store_sk#18, ave#28] +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] +Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) ColumnarToRow [codegen id : 7] +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] + +(35) HashAggregate [codegen id : 7] +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] +Keys [2]: [ss_store_sk#19, ss_item_sk#18] +Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#20))#26] +Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#26,17,2) AS revenue#27] + +(36) HashAggregate [codegen id : 7] +Input [2]: [ss_store_sk#19, revenue#27] +Keys [1]: [ss_store_sk#19] +Functions [1]: [partial_avg(revenue#27)] +Aggregate Attributes [2]: [sum#28, count#29] +Results [3]: [ss_store_sk#19, sum#30, count#31] + +(37) RowToColumnar +Input [3]: [ss_store_sk#19, sum#30, count#31] + +(38) CometColumnarExchange +Input [3]: [ss_store_sk#19, sum#30, count#31] +Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(39) ColumnarToRow [codegen id : 8] +Input [3]: [ss_store_sk#19, sum#30, count#31] + +(40) HashAggregate [codegen id : 8] +Input [3]: [ss_store_sk#19, sum#30, count#31] +Keys [1]: [ss_store_sk#19] +Functions [1]: [avg(revenue#27)] +Aggregate Attributes [1]: [avg(revenue#27)#32] +Results [2]: [ss_store_sk#19, avg(revenue#27)#32 AS ave#33] + +(41) Filter [codegen id : 8] +Input [2]: [ss_store_sk#19, ave#33] +Condition : isnotnull(ave#33) + +(42) BroadcastExchange +Input [2]: [ss_store_sk#19, ave#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 8] +(43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#18] +Right keys [1]: [ss_store_sk#19] Join type: Inner -Join condition: (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#28)) +Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) -(42) Project [codegen id : 8] -Output [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16, ss_store_sk#18, ave#28] +(44) Project [codegen id : 9] +Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17, ss_store_sk#19, ave#33] -(43) TakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#16] +(45) TakeOrderedAndProject +Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] +Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#29] +Output [2]: [d_date_sk#8, d_month_seq#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#29] -Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1176)) AND (d_month_seq#29 <= 1187)) AND isnotnull(d_date_sk#8)) +(47) CometFilter +Input [2]: [d_date_sk#8, d_month_seq#34] +Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_month_seq#34 <= 1187)) AND isnotnull(d_date_sk#8)) -(46) CometProject -Input [2]: [d_date_sk#8, d_month_seq#29] +(48) CometProject +Input [2]: [d_date_sk#8, d_month_seq#34] Arguments: [d_date_sk#8], [d_date_sk#8] -(47) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(48) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index 5a65234a30..880e68a392 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (8) + WholeStageCodegen (9) Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] @@ -13,10 +13,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes InputAdapter BroadcastExchange #1 WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [revenue] - CometHashAggregate [ss_store_sk,ss_item_sk,sum] + Filter [revenue] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ss_store_sk,ss_item_sk] #2 RowToColumnar WholeStageCodegen (2) @@ -46,24 +46,28 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [ave] - CometHashAggregate [ss_store_sk,sum,count] + WholeStageCodegen (8) + Filter [ave] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ss_store_sk] #6 - CometHashAggregate [ss_store_sk,revenue] - CometHashAggregate [ss_store_sk,ss_item_sk,sum] - CometColumnarExchange [ss_store_sk,ss_item_sk] #7 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_store_sk,ss_item_sk] #7 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index f4614e57bf..0189e14e7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -1,59 +1,62 @@ == Physical Plan == -* ColumnarToRow (55) -+- CometTakeOrderedAndProject (54) - +- CometHashAggregate (53) - +- CometColumnarExchange (52) - +- CometHashAggregate (51) - +- CometUnion (50) - :- CometHashAggregate (30) - : +- CometColumnarExchange (29) - : +- RowToColumnar (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.time_dim (13) - : +- BroadcastExchange (24) - : +- * ColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.ship_mode (20) - +- CometHashAggregate (49) - +- CometColumnarExchange (48) - +- RowToColumnar (47) - +- * HashAggregate (46) - +- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (36) - : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : :- * ColumnarToRow (33) - : : : : +- CometFilter (32) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : : +- ReusedExchange (34) - : : +- ReusedExchange (37) - : +- ReusedExchange (40) - +- ReusedExchange (43) +TakeOrderedAndProject (58) ++- * HashAggregate (57) + +- * ColumnarToRow (56) + +- CometColumnarExchange (55) + +- RowToColumnar (54) + +- * HashAggregate (53) + +- Union (52) + :- * HashAggregate (31) + : +- * ColumnarToRow (30) + : +- CometColumnarExchange (29) + : +- RowToColumnar (28) + : +- * HashAggregate (27) + : +- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.warehouse (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (17) + : : +- * ColumnarToRow (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.time_dim (13) + : +- BroadcastExchange (24) + : +- * ColumnarToRow (23) + : +- CometProject (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.ship_mode (20) + +- * HashAggregate (51) + +- * ColumnarToRow (50) + +- CometColumnarExchange (49) + +- RowToColumnar (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin Inner BuildRight (45) + :- * Project (43) + : +- * BroadcastHashJoin Inner BuildRight (42) + : :- * Project (40) + : : +- * BroadcastHashJoin Inner BuildRight (39) + : : :- * Project (37) + : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : :- * ColumnarToRow (34) + : : : : +- CometFilter (33) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (32) + : : : +- ReusedExchange (35) + : : +- ReusedExchange (38) + : +- ReusedExchange (41) + +- ReusedExchange (44) (unknown) Scan parquet spark_catalog.default.web_sales @@ -99,7 +102,7 @@ Join condition: None Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -(10) ReusedExchange [Reuses operator id: 59] +(10) ReusedExchange [Reuses operator id: 62] Output [3]: [d_date_sk#16, d_year#17, d_moy#18] (11) BroadcastHashJoin [codegen id : 5] @@ -190,129 +193,144 @@ Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometHashAggregate +(30) ColumnarToRow [codegen id : 6] +Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] + +(31) HashAggregate [codegen id : 6] Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17] Functions [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142] +Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#143, d_year#17 AS year#144, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119 AS jan_sales#145, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120 AS feb_sales#146, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121 AS mar_sales#147, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122 AS apr_sales#148, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123 AS may_sales#149, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124 AS jun_sales#150, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125 AS jul_sales#151, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126 AS aug_sales#152, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127 AS sep_sales#153, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128 AS oct_sales#154, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129 AS nov_sales#155, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130 AS dec_sales#156, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131 AS jan_net#157, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132 AS feb_net#158, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133 AS mar_net#159, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134 AS apr_net#160, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135 AS may_net#161, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136 AS jun_net#162, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137 AS jul_net#163, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138 AS aug_net#164, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139 AS sep_net#165, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140 AS oct_net#166, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141 AS nov_net#167, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142 AS dec_net#168] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_warehouse_sk#121, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125] +Output [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#125), dynamicpruningexpression(cs_sold_date_sk#125 IN dynamicpruning#126)] +PartitionFilters: [isnotnull(cs_sold_date_sk#175), dynamicpruningexpression(cs_sold_date_sk#175 IN dynamicpruning#176)] PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(32) CometFilter -Input [7]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_warehouse_sk#121, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125] -Condition : ((isnotnull(cs_warehouse_sk#121) AND isnotnull(cs_sold_time_sk#119)) AND isnotnull(cs_ship_mode_sk#120)) +(33) CometFilter +Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] +Condition : ((isnotnull(cs_warehouse_sk#171) AND isnotnull(cs_sold_time_sk#169)) AND isnotnull(cs_ship_mode_sk#170)) -(33) ColumnarToRow [codegen id : 10] -Input [7]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_warehouse_sk#121, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125] +(34) ColumnarToRow [codegen id : 11] +Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] -(34) ReusedExchange [Reuses operator id: 7] -Output [7]: [w_warehouse_sk#127, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133] +(35) ReusedExchange [Reuses operator id: 7] +Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -(35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_warehouse_sk#121] -Right keys [1]: [w_warehouse_sk#127] +(36) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_warehouse_sk#171] +Right keys [1]: [w_warehouse_sk#177] Join type: Inner Join condition: None -(36) Project [codegen id : 10] -Output [12]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133] -Input [14]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_warehouse_sk#121, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125, w_warehouse_sk#127, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133] +(37) Project [codegen id : 11] +Output [12]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] +Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -(37) ReusedExchange [Reuses operator id: 59] -Output [3]: [d_date_sk#134, d_year#135, d_moy#136] +(38) ReusedExchange [Reuses operator id: 62] +Output [3]: [d_date_sk#184, d_year#185, d_moy#186] -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#125] -Right keys [1]: [d_date_sk#134] +(39) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_date_sk#175] +Right keys [1]: [d_date_sk#184] Join type: Inner Join condition: None -(39) Project [codegen id : 10] -Output [13]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136] -Input [15]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, cs_sold_date_sk#125, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_date_sk#134, d_year#135, d_moy#136] +(40) Project [codegen id : 11] +Output [13]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +Input [15]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] -(40) ReusedExchange [Reuses operator id: 17] -Output [1]: [t_time_sk#137] +(41) ReusedExchange [Reuses operator id: 17] +Output [1]: [t_time_sk#187] -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_time_sk#119] -Right keys [1]: [t_time_sk#137] +(42) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_time_sk#169] +Right keys [1]: [t_time_sk#187] Join type: Inner Join condition: None -(42) Project [codegen id : 10] -Output [12]: [cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136] -Input [14]: [cs_sold_time_sk#119, cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136, t_time_sk#137] +(43) Project [codegen id : 11] +Output [12]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] -(43) ReusedExchange [Reuses operator id: 24] -Output [1]: [sm_ship_mode_sk#138] +(44) ReusedExchange [Reuses operator id: 24] +Output [1]: [sm_ship_mode_sk#188] -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_ship_mode_sk#120] -Right keys [1]: [sm_ship_mode_sk#138] +(45) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_ship_mode_sk#170] +Right keys [1]: [sm_ship_mode_sk#188] Join type: Inner Join condition: None -(45) Project [codegen id : 10] -Output [11]: [cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136] -Input [13]: [cs_ship_mode_sk#120, cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136, sm_ship_mode_sk#138] +(46) Project [codegen id : 11] +Output [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +Input [13]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] + +(47) HashAggregate [codegen id : 11] +Input [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] +Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236] +Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] + +(48) RowToColumnar +Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -(46) HashAggregate [codegen id : 10] -Input [11]: [cs_quantity#122, cs_sales_price#123, cs_net_paid_inc_tax#124, w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, d_moy#136] -Keys [7]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135] -Functions [24]: [partial_sum(CASE WHEN (d_moy#136 = 1) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 2) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 3) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 4) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 5) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 6) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 7) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 8) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 9) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 10) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 11) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 12) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 1) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 2) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 3) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 4) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 5) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 6) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 7) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 8) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 9) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 10) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 11) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#136 = 12) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] -Results [55]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234] +(49) CometColumnarExchange +Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] +Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(47) RowToColumnar -Input [55]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234] +(50) ColumnarToRow [codegen id : 12] +Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -(48) CometColumnarExchange -Input [55]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234] -Arguments: hashpartitioning(w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(51) HashAggregate [codegen id : 12] +Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] +Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] +Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308] +Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, DHL,BARIAN AS ship_carriers#309, d_year#185 AS year#310, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285 AS jan_sales#311, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286 AS feb_sales#312, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287 AS mar_sales#313, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288 AS apr_sales#314, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289 AS may_sales#315, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290 AS jun_sales#316, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291 AS jul_sales#317, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292 AS aug_sales#318, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293 AS sep_sales#319, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294 AS oct_sales#320, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295 AS nov_sales#321, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296 AS dec_sales#322, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297 AS jan_net#323, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298 AS feb_net#324, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299 AS mar_net#325, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300 AS apr_net#326, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301 AS may_net#327, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302 AS jun_net#328, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303 AS jul_net#329, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304 AS aug_net#330, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305 AS sep_net#331, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306 AS oct_net#332, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307 AS nov_net#333, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308 AS dec_net#334] -(49) CometHashAggregate -Input [55]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234] -Keys [7]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, d_year#135] -Functions [24]: [sum(CASE WHEN (d_moy#136 = 1) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 2) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 3) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 4) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 5) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 6) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 7) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 8) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 9) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 10) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 11) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 12) THEN (cs_sales_price#123 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 1) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 2) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 3) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 4) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 5) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 6) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 7) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 8) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 9) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 10) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 11) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#136 = 12) THEN (cs_net_paid_inc_tax#124 * cast(cs_quantity#122 as decimal(10,0))) ELSE 0.00 END)] +(52) Union -(50) CometUnion -Child 0 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, jan_sales#237, feb_sales#238, mar_sales#239, apr_sales#240, may_sales#241, jun_sales#242, jul_sales#243, aug_sales#244, sep_sales#245, oct_sales#246, nov_sales#247, dec_sales#248, jan_net#249, feb_net#250, mar_net#251, apr_net#252, may_net#253, jun_net#254, jul_net#255, aug_net#256, sep_net#257, oct_net#258, nov_net#259, dec_net#260] -Child 1 Input [32]: [w_warehouse_name#128, w_warehouse_sq_ft#129, w_city#130, w_county#131, w_state#132, w_country#133, ship_carriers#261, year#262, jan_sales#263, feb_sales#264, mar_sales#265, apr_sales#266, may_sales#267, jun_sales#268, jul_sales#269, aug_sales#270, sep_sales#271, oct_sales#272, nov_sales#273, dec_sales#274, jan_net#275, feb_net#276, mar_net#277, apr_net#278, may_net#279, jun_net#280, jul_net#281, aug_net#282, sep_net#283, oct_net#284, nov_net#285, dec_net#286] +(53) HashAggregate [codegen id : 13] +Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#145, feb_sales#146, mar_sales#147, apr_sales#148, may_sales#149, jun_sales#150, jul_sales#151, aug_sales#152, sep_sales#153, oct_sales#154, nov_sales#155, dec_sales#156, jan_net#157, feb_net#158, mar_net#159, apr_net#160, may_net#161, jun_net#162, jul_net#163, aug_net#164, sep_net#165, oct_net#166, nov_net#167, dec_net#168] +Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] +Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial_sum(mar_sales#147), partial_sum(apr_sales#148), partial_sum(may_sales#149), partial_sum(jun_sales#150), partial_sum(jul_sales#151), partial_sum(aug_sales#152), partial_sum(sep_sales#153), partial_sum(oct_sales#154), partial_sum(nov_sales#155), partial_sum(dec_sales#156), partial_sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#157), partial_sum(feb_net#158), partial_sum(mar_net#159), partial_sum(apr_net#160), partial_sum(may_net#161), partial_sum(jun_net#162), partial_sum(jul_net#163), partial_sum(aug_net#164), partial_sum(sep_net#165), partial_sum(oct_net#166), partial_sum(nov_net#167), partial_sum(dec_net#168)] +Aggregate Attributes [72]: [sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384, sum#385, isEmpty#386, sum#387, isEmpty#388, sum#389, isEmpty#390, sum#391, isEmpty#392, sum#393, isEmpty#394, sum#395, isEmpty#396, sum#397, isEmpty#398, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406] +Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -(51) CometHashAggregate -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, jan_sales#237, feb_sales#238, mar_sales#239, apr_sales#240, may_sales#241, jun_sales#242, jul_sales#243, aug_sales#244, sep_sales#245, oct_sales#246, nov_sales#247, dec_sales#248, jan_net#249, feb_net#250, mar_net#251, apr_net#252, may_net#253, jun_net#254, jul_net#255, aug_net#256, sep_net#257, oct_net#258, nov_net#259, dec_net#260] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236] -Functions [36]: [partial_sum(jan_sales#237), partial_sum(feb_sales#238), partial_sum(mar_sales#239), partial_sum(apr_sales#240), partial_sum(may_sales#241), partial_sum(jun_sales#242), partial_sum(jul_sales#243), partial_sum(aug_sales#244), partial_sum(sep_sales#245), partial_sum(oct_sales#246), partial_sum(nov_sales#247), partial_sum(dec_sales#248), partial_sum((jan_sales#237 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#238 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#239 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#240 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#241 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#242 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#243 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#244 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#245 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#246 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#247 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#248 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#249), partial_sum(feb_net#250), partial_sum(mar_net#251), partial_sum(apr_net#252), partial_sum(may_net#253), partial_sum(jun_net#254), partial_sum(jul_net#255), partial_sum(aug_net#256), partial_sum(sep_net#257), partial_sum(oct_net#258), partial_sum(nov_net#259), partial_sum(dec_net#260)] +(54) RowToColumnar +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -(52) CometColumnarExchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, sum#287, isEmpty#288, sum#289, isEmpty#290, sum#291, isEmpty#292, sum#293, isEmpty#294, sum#295, isEmpty#296, sum#297, isEmpty#298, sum#299, isEmpty#300, sum#301, isEmpty#302, sum#303, isEmpty#304, sum#305, isEmpty#306, sum#307, isEmpty#308, sum#309, isEmpty#310, sum#311, isEmpty#312, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(55) CometColumnarExchange +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(53) CometHashAggregate -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, sum#287, isEmpty#288, sum#289, isEmpty#290, sum#291, isEmpty#292, sum#293, isEmpty#294, sum#295, isEmpty#296, sum#297, isEmpty#298, sum#299, isEmpty#300, sum#301, isEmpty#302, sum#303, isEmpty#304, sum#305, isEmpty#306, sum#307, isEmpty#308, sum#309, isEmpty#310, sum#311, isEmpty#312, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236] -Functions [36]: [sum(jan_sales#237), sum(feb_sales#238), sum(mar_sales#239), sum(apr_sales#240), sum(may_sales#241), sum(jun_sales#242), sum(jul_sales#243), sum(aug_sales#244), sum(sep_sales#245), sum(oct_sales#246), sum(nov_sales#247), sum(dec_sales#248), sum((jan_sales#237 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#238 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#239 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#240 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#241 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#242 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#243 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#244 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#245 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#246 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#247 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#248 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#249), sum(feb_net#250), sum(mar_net#251), sum(apr_net#252), sum(may_net#253), sum(jun_net#254), sum(jul_net#255), sum(aug_net#256), sum(sep_net#257), sum(oct_net#258), sum(nov_net#259), sum(dec_net#260)] +(56) ColumnarToRow [codegen id : 14] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -(54) CometTakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, jan_sales#359, feb_sales#360, mar_sales#361, apr_sales#362, may_sales#363, jun_sales#364, jul_sales#365, aug_sales#366, sep_sales#367, oct_sales#368, nov_sales#369, dec_sales#370, jan_sales_per_sq_foot#371, feb_sales_per_sq_foot#372, mar_sales_per_sq_foot#373, apr_sales_per_sq_foot#374, may_sales_per_sq_foot#375, jun_sales_per_sq_foot#376, jul_sales_per_sq_foot#377, aug_sales_per_sq_foot#378, sep_sales_per_sq_foot#379, oct_sales_per_sq_foot#380, nov_sales_per_sq_foot#381, dec_sales_per_sq_foot#382, jan_net#383, feb_net#384, mar_net#385, apr_net#386, may_net#387, jun_net#388, jul_net#389, aug_net#390, sep_net#391, oct_net#392, nov_net#393, dec_net#394] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#14,w_country#15,ship_carriers#235,year#236,jan_sales#359,feb_sales#360,mar_sales#361,apr_sales#362,may_sales#363,jun_sales#364,jul_sales#365,aug_sales#366,sep_sales#367,oct_sales#368,nov_sales#369,dec_sales#370,jan_sales_per_sq_foot#371,feb_sales_per_sq_foot#372,mar_sales_per_sq_foot#373,apr_sales_per_sq_foot#374,may_sales_per_sq_foot#375,jun_sales_per_sq_foot#376,jul_sales_per_sq_foot#377,aug_sales_per_sq_foot#378,sep_sales_per_sq_foot#379,oct_sales_per_sq_foot#380,nov_sales_per_sq_foot#381,dec_sales_per_sq_foot#382,jan_net#383,feb_net#384,mar_net#385,apr_net#386,may_net#387,jun_net#388,jul_net#389,aug_net#390,sep_net#391,oct_net#392,nov_net#393,dec_net#394]), 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, jan_sales#359, feb_sales#360, mar_sales#361, apr_sales#362, may_sales#363, jun_sales#364, jul_sales#365, aug_sales#366, sep_sales#367, oct_sales#368, nov_sales#369, dec_sales#370, jan_sales_per_sq_foot#371, feb_sales_per_sq_foot#372, mar_sales_per_sq_foot#373, apr_sales_per_sq_foot#374, may_sales_per_sq_foot#375, jun_sales_per_sq_foot#376, jul_sales_per_sq_foot#377, aug_sales_per_sq_foot#378, sep_sales_per_sq_foot#379, oct_sales_per_sq_foot#380, nov_sales_per_sq_foot#381, dec_sales_per_sq_foot#382, jan_net#383, feb_net#384, mar_net#385, apr_net#386, may_net#387, jun_net#388, jul_net#389, aug_net#390, sep_net#391, oct_net#392, nov_net#393, dec_net#394] +(57) HashAggregate [codegen id : 14] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] +Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] +Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#157), sum(feb_net#158), sum(mar_net#159), sum(apr_net#160), sum(may_net#161), sum(jun_net#162), sum(jul_net#163), sum(aug_net#164), sum(sep_net#165), sum(oct_net#166), sum(nov_net#167), sum(dec_net#168)] +Aggregate Attributes [36]: [sum(jan_sales#145)#479, sum(feb_sales#146)#480, sum(mar_sales#147)#481, sum(apr_sales#148)#482, sum(may_sales#149)#483, sum(jun_sales#150)#484, sum(jul_sales#151)#485, sum(aug_sales#152)#486, sum(sep_sales#153)#487, sum(oct_sales#154)#488, sum(nov_sales#155)#489, sum(dec_sales#156)#490, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502, sum(jan_net#157)#503, sum(feb_net#158)#504, sum(mar_net#159)#505, sum(apr_net#160)#506, sum(may_net#161)#507, sum(jun_net#162)#508, sum(jul_net#163)#509, sum(aug_net#164)#510, sum(sep_net#165)#511, sum(oct_net#166)#512, sum(nov_net#167)#513, sum(dec_net#168)#514] +Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum(jan_sales#145)#479 AS jan_sales#515, sum(feb_sales#146)#480 AS feb_sales#516, sum(mar_sales#147)#481 AS mar_sales#517, sum(apr_sales#148)#482 AS apr_sales#518, sum(may_sales#149)#483 AS may_sales#519, sum(jun_sales#150)#484 AS jun_sales#520, sum(jul_sales#151)#485 AS jul_sales#521, sum(aug_sales#152)#486 AS aug_sales#522, sum(sep_sales#153)#487 AS sep_sales#523, sum(oct_sales#154)#488 AS oct_sales#524, sum(nov_sales#155)#489 AS nov_sales#525, sum(dec_sales#156)#490 AS dec_sales#526, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491 AS jan_sales_per_sq_foot#527, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492 AS feb_sales_per_sq_foot#528, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493 AS mar_sales_per_sq_foot#529, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494 AS apr_sales_per_sq_foot#530, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495 AS may_sales_per_sq_foot#531, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496 AS jun_sales_per_sq_foot#532, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497 AS jul_sales_per_sq_foot#533, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498 AS aug_sales_per_sq_foot#534, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499 AS sep_sales_per_sq_foot#535, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500 AS oct_sales_per_sq_foot#536, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501 AS nov_sales_per_sq_foot#537, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502 AS dec_sales_per_sq_foot#538, sum(jan_net#157)#503 AS jan_net#539, sum(feb_net#158)#504 AS feb_net#540, sum(mar_net#159)#505 AS mar_net#541, sum(apr_net#160)#506 AS apr_net#542, sum(may_net#161)#507 AS may_net#543, sum(jun_net#162)#508 AS jun_net#544, sum(jul_net#163)#509 AS jul_net#545, sum(aug_net#164)#510 AS aug_net#546, sum(sep_net#165)#511 AS sep_net#547, sum(oct_net#166)#512 AS oct_net#548, sum(nov_net#167)#513 AS nov_net#549, sum(dec_net#168)#514 AS dec_net#550] -(55) ColumnarToRow [codegen id : 11] -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#235, year#236, jan_sales#359, feb_sales#360, mar_sales#361, apr_sales#362, may_sales#363, jun_sales#364, jul_sales#365, aug_sales#366, sep_sales#367, oct_sales#368, nov_sales#369, dec_sales#370, jan_sales_per_sq_foot#371, feb_sales_per_sq_foot#372, mar_sales_per_sq_foot#373, apr_sales_per_sq_foot#374, may_sales_per_sq_foot#375, jun_sales_per_sq_foot#376, jul_sales_per_sq_foot#377, aug_sales_per_sq_foot#378, sep_sales_per_sq_foot#379, oct_sales_per_sq_foot#380, nov_sales_per_sq_foot#381, dec_sales_per_sq_foot#382, jan_net#383, feb_net#384, mar_net#385, apr_net#386, may_net#387, jun_net#388, jul_net#389, aug_net#390, sep_net#391, oct_net#392, nov_net#393, dec_net#394] +(58) TakeOrderedAndProject +Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] +Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (59) -+- * ColumnarToRow (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.date_dim (56) +BroadcastExchange (62) ++- * ColumnarToRow (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.date_dim (59) (unknown) Scan parquet spark_catalog.default.date_dim @@ -322,17 +340,17 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter +(60) CometFilter Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(58) ColumnarToRow [codegen id : 1] +(61) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -(59) BroadcastExchange +(62) BroadcastExchange Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#125 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#175 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index 025a2b9d4d..3c5f5bd2f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -1,83 +1,92 @@ -WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] +TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] + WholeStageCodegen (14) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - CometUnion - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + RowToColumnar + WholeStageCodegen (13) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_time,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) + BroadcastExchange #6 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometProject [sm_ship_mode_sk] + CometFilter [sm_carrier,sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + WholeStageCodegen (12) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 + RowToColumnar + WholeStageCodegen (11) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_sold_time_sk,t_time_sk] + Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #3 + InputAdapter + ReusedExchange [t_time_sk] #5 InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - ReusedExchange [t_time_sk] #5 - InputAdapter - ReusedExchange [sm_ship_mode_sk] #6 + ReusedExchange [sm_ship_mode_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index fc5ae3bc19..f84c06d92d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -1,33 +1,35 @@ == Physical Plan == -TakeOrderedAndProject (29) -+- * Filter (28) - +- Window (27) - +- * ColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- CometHashAggregate (23) - +- CometColumnarExchange (22) - +- RowToColumnar (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.item (13) +TakeOrderedAndProject (31) ++- * Filter (30) + +- Window (29) + +- * ColumnarToRow (28) + +- CometSort (27) + +- CometColumnarExchange (26) + +- RowToColumnar (25) + +- * HashAggregate (24) + +- * ColumnarToRow (23) + +- CometColumnarExchange (22) + +- RowToColumnar (21) + +- * HashAggregate (20) + +- * Expand (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.store (7) + +- BroadcastExchange (16) + +- * ColumnarToRow (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.item (13) (unknown) Scan parquet spark_catalog.default.store_sales @@ -45,7 +47,7 @@ Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 34] +(4) ReusedExchange [Reuses operator id: 36] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (5) BroadcastHashJoin [codegen id : 4] @@ -132,63 +134,71 @@ Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22 Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] Arguments: hashpartitioning(i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometHashAggregate +(23) ColumnarToRow [codegen id : 5] +Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] + +(24) HashAggregate [codegen id : 5] Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31] +Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31 AS sumsales#32] + +(25) RowToColumnar +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -(24) CometColumnarExchange -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31] +(26) CometColumnarExchange +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) CometSort -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31] -Arguments: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31], [i_category#18 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST] +(27) CometSort +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] +Arguments: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32], [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST] -(26) ColumnarToRow [codegen id : 5] -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31] +(28) ColumnarToRow [codegen id : 6] +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -(27) Window -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31] -Arguments: [rank(sumsales#31) windowspecdefinition(i_category#18, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [i_category#18], [sumsales#31 DESC NULLS LAST] +(29) Window +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] +Arguments: [rank(sumsales#32) windowspecdefinition(i_category#18, sumsales#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#33], [i_category#18], [sumsales#32 DESC NULLS LAST] -(28) Filter [codegen id : 6] -Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31, rk#32] -Condition : (rk#32 <= 100) +(30) Filter [codegen id : 7] +Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] +Condition : (rk#33 <= 100) -(29) TakeOrderedAndProject -Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31, rk#32] -Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_qoy#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#32 ASC NULLS FIRST], [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#31, rk#32] +(31) TakeOrderedAndProject +Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] +Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_qoy#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, sumsales#32 ASC NULLS FIRST, rk#33 ASC NULLS FIRST], [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (34) -+- * ColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.date_dim (30) +BroadcastExchange (36) ++- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) (unknown) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#33, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(31) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#33, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1200)) AND (d_month_seq#33 <= 1211)) AND isnotnull(d_date_sk#7)) +(33) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#7)) -(32) CometProject -Input [5]: [d_date_sk#7, d_month_seq#33, d_year#8, d_moy#9, d_qoy#10] +(34) CometProject +Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(33) ColumnarToRow [codegen id : 1] +(35) ColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(34) BroadcastExchange +(36) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index 3fd090f51e..5e232e682d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -1,50 +1,54 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (6) + WholeStageCodegen (7) Filter [rk] InputAdapter Window [sumsales,i_category] - WholeStageCodegen (5) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometSort [i_category,sumsales] CometColumnarExchange [i_category] #1 - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index 5b01bcc0da..95e98afea0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -4,8 +4,8 @@ TakeOrderedAndProject (41) +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (37) : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * ColumnarToRow (31) - : : +- CometHashAggregate (30) + : :- * HashAggregate (31) + : : +- * ColumnarToRow (30) : : +- CometColumnarExchange (29) : : +- RowToColumnar (28) : : +- * HashAggregate (27) @@ -176,58 +176,60 @@ Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum# Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometHashAggregate +(30) ColumnarToRow [codegen id : 8] +Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] + +(31) HashAggregate [codegen id : 8] Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] - -(31) ColumnarToRow [codegen id : 8] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#25, extended_price#26, list_price#27, extended_tax#28] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#25, sum(UnscaledValue(ss_ext_list_price#7))#26, sum(UnscaledValue(ss_ext_tax#8))#27] +Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#18 AS bought_city#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#25,17,2) AS extended_price#29, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#26,17,2) AS list_price#30, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#27,17,2) AS extended_tax#31] (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] +Output [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (33) CometFilter -Input [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] -Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#30)) +Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) (34) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] +Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] (35) BroadcastExchange -Input [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] +Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] (36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None (37) Project [codegen id : 8] -Output [8]: [ss_ticket_number#5, bought_city#25, extended_price#26, list_price#27, extended_tax#28, c_current_addr_sk#30, c_first_name#31, c_last_name#32] -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#25, extended_price#26, list_price#27, extended_tax#28, c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] +Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] (38) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#33, ca_city#34] +Output [2]: [ca_address_sk#36, ca_city#37] (39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#30] -Right keys [1]: [ca_address_sk#33] +Left keys [1]: [c_current_addr_sk#33] +Right keys [1]: [ca_address_sk#36] Join type: Inner -Join condition: NOT (ca_city#34 = bought_city#25) +Join condition: NOT (ca_city#37 = bought_city#28) (40) Project [codegen id : 8] -Output [8]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#25, ss_ticket_number#5, extended_price#26, extended_tax#28, list_price#27] -Input [10]: [ss_ticket_number#5, bought_city#25, extended_price#26, list_price#27, extended_tax#28, c_current_addr_sk#30, c_first_name#31, c_last_name#32, ca_address_sk#33, ca_city#34] +Output [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] +Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35, ca_address_sk#36, ca_city#37] (41) TakeOrderedAndProject -Input [8]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#25, ss_ticket_number#5, extended_price#26, extended_tax#28, list_price#27] -Arguments: 100, [c_last_name#32 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#32, c_first_name#31, ca_city#34, bought_city#25, ss_ticket_number#5, extended_price#26, extended_tax#28, list_price#27] +Input [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] +Arguments: 100, [c_last_name#35 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] ===== Subqueries ===== @@ -240,18 +242,18 @@ BroadcastExchange (46) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#35, d_dom#36] +Output [3]: [d_date_sk#11, d_year#38, d_dom#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct (43) CometFilter -Input [3]: [d_date_sk#11, d_year#35, d_dom#36] -Condition : ((((isnotnull(d_dom#36) AND (d_dom#36 >= 1)) AND (d_dom#36 <= 2)) AND d_year#35 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#11, d_year#38, d_dom#39] +Condition : ((((isnotnull(d_dom#39) AND (d_dom#39 >= 1)) AND (d_dom#39 <= 2)) AND d_year#38 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) (44) CometProject -Input [3]: [d_date_sk#11, d_year#35, d_dom#36] +Input [3]: [d_date_sk#11, d_year#38, d_dom#39] Arguments: [d_date_sk#11], [d_date_sk#11] (45) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt index a4396d582b..cad5e40b6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -4,9 +4,9 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 RowToColumnar WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index 6c289ac77b..9b5785dd62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- * ColumnarToRow (42) +- CometColumnarExchange (41) +- RowToColumnar (40) +- * HashAggregate (39) @@ -236,17 +236,19 @@ Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purch Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(42) CometHashAggregate +(42) ColumnarToRow [codegen id : 10] +Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] + +(43) HashAggregate [codegen id : 10] Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] Keys [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#26] +Results [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, count(1)#26 AS cnt1#27, cd_purchase_estimate#22, count(1)#26 AS cnt2#28, cd_credit_rating#23, count(1)#26 AS cnt3#29] -(43) CometTakeOrderedAndProject -Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#26, cd_purchase_estimate#22, cnt2#27, cd_credit_rating#23, cnt3#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#19 ASC NULLS FIRST,cd_marital_status#20 ASC NULLS FIRST,cd_education_status#21 ASC NULLS FIRST,cd_purchase_estimate#22 ASC NULLS FIRST,cd_credit_rating#23 ASC NULLS FIRST], output=[cd_gender#19,cd_marital_status#20,cd_education_status#21,cnt1#26,cd_purchase_estimate#22,cnt2#27,cd_credit_rating#23,cnt3#28]), 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#26, cd_purchase_estimate#22, cnt2#27, cd_credit_rating#23, cnt3#28] - -(44) ColumnarToRow [codegen id : 10] -Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#26, cd_purchase_estimate#22, cnt2#27, cd_credit_rating#23, cnt3#28] +(44) TakeOrderedAndProject +Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] +Arguments: 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] ===== Subqueries ===== @@ -259,18 +261,18 @@ BroadcastExchange (49) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#29, d_moy#30] +Output [3]: [d_date_sk#7, d_year#30, d_moy#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct (46) CometFilter -Input [3]: [d_date_sk#7, d_year#29, d_moy#30] -Condition : (((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 >= 4)) AND (d_moy#30 <= 6)) AND isnotnull(d_date_sk#7)) +Input [3]: [d_date_sk#7, d_year#30, d_moy#31] +Condition : (((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2001)) AND (d_moy#31 >= 4)) AND (d_moy#31 <= 6)) AND isnotnull(d_date_sk#7)) (47) CometProject -Input [3]: [d_date_sk#7, d_year#29, d_moy#30] +Input [3]: [d_date_sk#7, d_year#30, d_moy#31] Arguments: [d_date_sk#7], [d_date_sk#7] (48) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index 410950aeb0..ad64fa5067 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] +TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] + WholeStageCodegen (10) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] + ColumnarToRow + InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 RowToColumnar WholeStageCodegen (9) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt index 31a4d6acdb..6d17275156 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (32) -+- CometTakeOrderedAndProject (31) - +- CometHashAggregate (30) +TakeOrderedAndProject (32) ++- * HashAggregate (31) + +- * ColumnarToRow (30) +- CometColumnarExchange (29) +- RowToColumnar (28) +- * HashAggregate (27) @@ -167,17 +167,19 @@ Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometHashAggregate +(30) ColumnarToRow [codegen id : 6] +Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] + +(31) HashAggregate [codegen id : 6] Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] Keys [1]: [i_item_id#16] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [4]: [avg(ss_quantity#4)#36, avg(UnscaledValue(ss_list_price#5))#37, avg(UnscaledValue(ss_coupon_amt#7))#38, avg(UnscaledValue(ss_sales_price#6))#39] +Results [5]: [i_item_id#16, avg(ss_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(ss_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(ss_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(ss_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] -(31) CometTakeOrderedAndProject -Input [5]: [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#16 ASC NULLS FIRST], output=[i_item_id#16,agg1#36,agg2#37,agg3#38,agg4#39]), 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] - -(32) ColumnarToRow [codegen id : 6] -Input [5]: [i_item_id#16, agg1#36, agg2#37, agg3#38, agg4#39] +(32) TakeOrderedAndProject +Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] +Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] ===== Subqueries ===== @@ -190,18 +192,18 @@ BroadcastExchange (37) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#40] +Output [2]: [d_date_sk#14, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (34) CometFilter -Input [2]: [d_date_sk#14, d_year#40] -Condition : ((isnotnull(d_year#40) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) (35) CometProject -Input [2]: [d_date_sk#14, d_year#40] +Input [2]: [d_date_sk#14, d_year#44] Arguments: [d_date_sk#14], [d_date_sk#14] (36) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt index 37d5a50ae0..eba10dc409 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] +TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] + WholeStageCodegen (6) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #1 RowToColumnar WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 8387487b4f..422c4bbd10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -1,50 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (46) -+- * Project (45) - +- Window (44) - +- * ColumnarToRow (43) - +- CometSort (42) - +- CometColumnarExchange (41) - +- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- RowToColumnar (38) - +- * HashAggregate (37) - +- * Expand (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (33) - +- * BroadcastHashJoin LeftSemi BuildRight (32) - :- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (31) - +- * Project (30) - +- * Filter (29) - +- Window (28) - +- * ColumnarToRow (27) - +- CometSort (26) - +- CometHashAggregate (25) - +- CometColumnarExchange (24) - +- RowToColumnar (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.store (13) - +- ReusedExchange (19) +TakeOrderedAndProject (48) ++- * Project (47) + +- Window (46) + +- * ColumnarToRow (45) + +- CometSort (44) + +- CometColumnarExchange (43) + +- RowToColumnar (42) + +- * HashAggregate (41) + +- * ColumnarToRow (40) + +- CometColumnarExchange (39) + +- RowToColumnar (38) + +- * HashAggregate (37) + +- * Expand (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- ReusedExchange (4) + +- BroadcastExchange (33) + +- * BroadcastHashJoin LeftSemi BuildRight (32) + :- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.store (7) + +- BroadcastExchange (31) + +- * Project (30) + +- * Filter (29) + +- Window (28) + +- * Sort (27) + +- * HashAggregate (26) + +- * ColumnarToRow (25) + +- CometColumnarExchange (24) + +- RowToColumnar (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildRight (20) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store_sales (10) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.store (13) + +- ReusedExchange (19) (unknown) Scan parquet spark_catalog.default.store_sales @@ -62,7 +64,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 8] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 51] +(4) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 8] @@ -132,7 +134,7 @@ Join condition: None Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] -(19) ReusedExchange [Reuses operator id: 51] +(19) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#15] (20) BroadcastHashJoin [codegen id : 4] @@ -159,29 +161,31 @@ Input [2]: [s_state#14, sum#17] Input [2]: [s_state#14, sum#17] Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(25) CometHashAggregate +(25) ColumnarToRow [codegen id : 5] +Input [2]: [s_state#14, sum#17] + +(26) HashAggregate [codegen id : 5] Input [2]: [s_state#14, sum#17] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] +Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] -(26) CometSort -Input [3]: [s_state#14, _w0#18, s_state#14] -Arguments: [s_state#14, _w0#18, s_state#14], [s_state#14 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] - -(27) ColumnarToRow [codegen id : 5] -Input [3]: [s_state#14, _w0#18, s_state#14] +(27) Sort [codegen id : 5] +Input [3]: [s_state#14, _w0#19, s_state#14] +Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 (28) Window -Input [3]: [s_state#14, _w0#18, s_state#14] -Arguments: [rank(_w0#18) windowspecdefinition(s_state#14, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#14], [_w0#18 DESC NULLS LAST] +Input [3]: [s_state#14, _w0#19, s_state#14] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] (29) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] -Condition : (ranking#19 <= 5) +Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] +Condition : (ranking#20 <= 5) (30) Project [codegen id : 6] Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] +Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] (31) BroadcastExchange Input [1]: [s_state#14] @@ -209,79 +213,87 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] (36) Expand [codegen id : 8] Input [3]: [ss_net_profit#2, s_state#8, s_county#7] -Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22] +Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] (37) HashAggregate [codegen id : 8] -Input [4]: [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22] -Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] +Input [4]: [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] +Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#23] -Results [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] +Aggregate Attributes [1]: [sum#24] +Results [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] (38) RowToColumnar -Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] +Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] (39) CometColumnarExchange -Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] -Arguments: hashpartitioning(s_state#20, s_county#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] +Arguments: hashpartitioning(s_state#21, s_county#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometHashAggregate -Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] -Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] +(40) ColumnarToRow [codegen id : 9] +Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] + +(41) HashAggregate [codegen id : 9] +Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] +Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#26] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS total_sum#27, s_state#21, s_county#22, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS lochierarchy#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS _w0#29, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS _w1#30, CASE WHEN (cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint) = 0) THEN s_state#21 END AS _w2#31] + +(42) RowToColumnar +Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] -(41) CometColumnarExchange -Input [7]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: hashpartitioning(_w1#28, _w2#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(43) CometColumnarExchange +Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] +Arguments: hashpartitioning(_w1#30, _w2#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(42) CometSort -Input [7]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29], [_w1#28 ASC NULLS FIRST, _w2#29 ASC NULLS FIRST, _w0#27 DESC NULLS LAST] +(44) CometSort +Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] +Arguments: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31], [_w1#30 ASC NULLS FIRST, _w2#31 ASC NULLS FIRST, _w0#29 DESC NULLS LAST] -(43) ColumnarToRow [codegen id : 9] -Input [7]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29] +(45) ColumnarToRow [codegen id : 10] +Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] -(44) Window -Input [7]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [rank(_w0#27) windowspecdefinition(_w1#28, _w2#29, _w0#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#30], [_w1#28, _w2#29], [_w0#27 DESC NULLS LAST] +(46) Window +Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] +Arguments: [rank(_w0#29) windowspecdefinition(_w1#30, _w2#31, _w0#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#32], [_w1#30, _w2#31], [_w0#29 DESC NULLS LAST] -(45) Project [codegen id : 10] -Output [5]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, rank_within_parent#30] -Input [8]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, _w0#27, _w1#28, _w2#29, rank_within_parent#30] +(47) Project [codegen id : 11] +Output [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] +Input [8]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31, rank_within_parent#32] -(46) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#20, s_county#21, lochierarchy#26, rank_within_parent#30] -Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#20 END ASC NULLS FIRST, rank_within_parent#30 ASC NULLS FIRST], [total_sum#25, s_state#20, s_county#21, lochierarchy#26, rank_within_parent#30] +(48) TakeOrderedAndProject +Input [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] +Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN s_state#21 END ASC NULLS FIRST, rank_within_parent#32 ASC NULLS FIRST], [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (51) -+- * ColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan parquet spark_catalog.default.date_dim (47) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#31] +Output [2]: [d_date_sk#5, d_month_seq#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#31] -Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1200)) AND (d_month_seq#31 <= 1211)) AND isnotnull(d_date_sk#5)) +(50) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#33] +Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1200)) AND (d_month_seq#33 <= 1211)) AND isnotnull(d_date_sk#5)) -(49) CometProject -Input [2]: [d_date_sk#5, d_month_seq#31] +(51) CometProject +Input [2]: [d_date_sk#5, d_month_seq#33] Arguments: [d_date_sk#5], [d_date_sk#5] -(50) ColumnarToRow [codegen id : 1] +(52) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(51) BroadcastExchange +(53) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index 53a93dc43f..1924ac21cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -1,76 +1,80 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (10) + WholeStageCodegen (11) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (9) + WholeStageCodegen (10) ColumnarToRow InputAdapter CometSort [_w1,_w2,_w0] CometColumnarExchange [_w1,_w2] #1 - CometHashAggregate [s_state,s_county,spark_grouping_id,sum] - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - BroadcastHashJoin [s_state,s_state] - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + RowToColumnar + WholeStageCodegen (9) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] + BroadcastExchange #4 + WholeStageCodegen (7) + BroadcastHashJoin [s_state,s_state] + ColumnarToRow InputAdapter - Window [_w0,s_state] - WholeStageCodegen (5) - ColumnarToRow + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + Project [s_state] + Filter [ranking] InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [s_state,sum] - CometColumnarExchange [s_state] #6 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow + Window [_w0,s_state] + WholeStageCodegen (5) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #6 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 172f460c8a..90ae1f6bfa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -1,44 +1,46 @@ == Physical Plan == -* ColumnarToRow (40) -+- CometSort (39) - +- CometColumnarExchange (38) - +- CometHashAggregate (37) - +- CometColumnarExchange (36) - +- RowToColumnar (35) - +- * HashAggregate (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildLeft (25) - : :- BroadcastExchange (5) - : : +- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.item (1) - : +- Union (24) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.web_sales (6) - : : +- ReusedExchange (9) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * ColumnarToRow (14) - : : : +- CometFilter (13) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : +- ReusedExchange (15) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * ColumnarToRow (20) - : : +- CometFilter (19) - : : +- CometScan parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (21) - +- BroadcastExchange (31) - +- * ColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.time_dim (27) +* ColumnarToRow (42) ++- CometSort (41) + +- CometColumnarExchange (40) + +- RowToColumnar (39) + +- * HashAggregate (38) + +- * ColumnarToRow (37) + +- CometColumnarExchange (36) + +- RowToColumnar (35) + +- * HashAggregate (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildLeft (25) + : :- BroadcastExchange (5) + : : +- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.item (1) + : +- Union (24) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.web_sales (6) + : : +- ReusedExchange (9) + : :- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * ColumnarToRow (14) + : : : +- CometFilter (13) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : +- ReusedExchange (15) + : +- * Project (23) + : +- * BroadcastHashJoin Inner BuildRight (22) + : :- * ColumnarToRow (20) + : : +- CometFilter (19) + : : +- CometScan parquet spark_catalog.default.store_sales (18) + : +- ReusedExchange (21) + +- BroadcastExchange (31) + +- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.time_dim (27) (unknown) Scan parquet spark_catalog.default.item @@ -78,7 +80,7 @@ Condition : (isnotnull(ws_item_sk#6) AND isnotnull(ws_sold_time_sk#5)) (8) ColumnarToRow [codegen id : 3] Input [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] -(9) ReusedExchange [Reuses operator id: 45] +(9) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#10] (10) BroadcastHashJoin [codegen id : 3] @@ -106,7 +108,7 @@ Condition : (isnotnull(cs_item_sk#15) AND isnotnull(cs_sold_time_sk#14)) (14) ColumnarToRow [codegen id : 5] Input [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] -(15) ReusedExchange [Reuses operator id: 45] +(15) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#19] (16) BroadcastHashJoin [codegen id : 5] @@ -134,7 +136,7 @@ Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) (20) ColumnarToRow [codegen id : 7] Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(21) ReusedExchange [Reuses operator id: 45] +(21) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#28] (22) BroadcastHashJoin [codegen id : 7] @@ -205,51 +207,59 @@ Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(37) CometHashAggregate +(37) ColumnarToRow [codegen id : 10] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] + +(38) HashAggregate [codegen id : 10] Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] Functions [1]: [sum(UnscaledValue(ext_price#11))] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#11))#38] +Results [5]: [i_brand_id#2 AS brand_id#39, i_brand#3 AS brand#40, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#11))#38,17,2) AS ext_price#41] + +(39) RowToColumnar +Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] -(38) CometColumnarExchange -Input [5]: [brand_id#38, brand#39, t_hour#33, t_minute#34, ext_price#40] -Arguments: rangepartitioning(ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(40) CometColumnarExchange +Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] +Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(39) CometSort -Input [5]: [brand_id#38, brand#39, t_hour#33, t_minute#34, ext_price#40] -Arguments: [brand_id#38, brand#39, t_hour#33, t_minute#34, ext_price#40], [ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST] +(41) CometSort +Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] +Arguments: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41], [ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST] -(40) ColumnarToRow [codegen id : 10] -Input [5]: [brand_id#38, brand#39, t_hour#33, t_minute#34, ext_price#40] +(42) ColumnarToRow [codegen id : 11] +Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#41, d_moy#42] +Output [3]: [d_date_sk#10, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#10, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 11)) AND (d_year#41 = 1999)) AND isnotnull(d_date_sk#10)) +(44) CometFilter +Input [3]: [d_date_sk#10, d_year#42, d_moy#43] +Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 11)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#10)) -(43) CometProject -Input [3]: [d_date_sk#10, d_year#41, d_moy#42] +(45) CometProject +Input [3]: [d_date_sk#10, d_year#42, d_moy#43] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(45) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index b5c6e96d6f..797f739ab0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -1,69 +1,73 @@ -WholeStageCodegen (10) +WholeStageCodegen (11) ColumnarToRow InputAdapter CometSort [ext_price,brand_id] CometColumnarExchange [ext_price,brand_id] #1 - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] - CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] - BroadcastHashJoin [time_sk,t_time_sk] - Project [i_brand_id,i_brand,ext_price,time_sk] - BroadcastHashJoin [i_item_sk,sold_item_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow + RowToColumnar + WholeStageCodegen (10) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 + RowToColumnar + WholeStageCodegen (9) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] + Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] + BroadcastHashJoin [time_sk,t_time_sk] + Project [i_brand_id,i_brand,ext_price,time_sk] + BroadcastHashJoin [i_item_sk,sold_item_sk] InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - InputAdapter - Union - WholeStageCodegen (3) - Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_sold_time_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (5) - Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_sold_time_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (7) - Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + InputAdapter + Union + WholeStageCodegen (3) + Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_sold_time_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (5) + Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_sold_time_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (7) + Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_sold_time_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_sold_time_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_meal_time,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index 38bf356802..ffdecea92e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt @@ -5,9 +5,9 @@ +- RowToColumnar (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) - :- * ColumnarToRow (26) - : +- CometFilter (25) - : +- CometHashAggregate (24) + :- * Filter (26) + : +- * HashAggregate (25) + : +- * ColumnarToRow (24) : +- CometColumnarExchange (23) : +- RowToColumnar (22) : +- * HashAggregate (21) @@ -143,59 +143,61 @@ Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometHashAggregate +(24) ColumnarToRow [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] + +(25) HashAggregate [codegen id : 6] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#16] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] -Condition : ((cnt#16 >= 1) AND (cnt#16 <= 5)) - -(26) ColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] +(26) Filter [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) (unknown) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (28) CometFilter -Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] -Condition : isnotnull(c_customer_sk#17) +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Condition : isnotnull(c_customer_sk#18) (29) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] (30) BroadcastExchange -Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#17] +Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None (32) Project [codegen id : 6] -Output [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16, c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] (33) RowToColumnar -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] (34) CometColumnarExchange -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] -Arguments: rangepartitioning(cnt#16 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] (35) CometSort -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] -Arguments: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16], [cnt#16 DESC NULLS LAST] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] (36) ColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== @@ -208,18 +210,18 @@ BroadcastExchange (41) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#22, d_dom#23] +Output [3]: [d_date_sk#7, d_year#23, d_dom#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct (38) CometFilter -Input [3]: [d_date_sk#7, d_year#22, d_dom#23] -Condition : ((((isnotnull(d_dom#23) AND (d_dom#23 >= 1)) AND (d_dom#23 <= 2)) AND d_year#22 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +Condition : ((((isnotnull(d_dom#24) AND (d_dom#24 >= 1)) AND (d_dom#24 <= 2)) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) (39) CometProject -Input [3]: [d_date_sk#7, d_year#22, d_dom#23] +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] Arguments: [d_date_sk#7], [d_date_sk#7] (40) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index 45a7796f70..da73b27ba1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -7,10 +7,10 @@ WholeStageCodegen (7) WholeStageCodegen (6) Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 RowToColumnar WholeStageCodegen (4) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index 431ca36ddf..54db51cb33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -5,9 +5,9 @@ TakeOrderedAndProject (79) :- * Project (58) : +- * BroadcastHashJoin Inner BuildRight (57) : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometHashAggregate (16) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * ColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- RowToColumnar (14) : : : +- * HashAggregate (13) @@ -24,8 +24,8 @@ TakeOrderedAndProject (79) : : : : +- CometScan parquet spark_catalog.default.store_sales (4) : : : +- ReusedExchange (10) : : +- BroadcastExchange (36) - : : +- * ColumnarToRow (35) - : : +- CometHashAggregate (34) + : : +- * HashAggregate (35) + : : +- * ColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- RowToColumnar (32) : : +- * HashAggregate (31) @@ -42,9 +42,9 @@ TakeOrderedAndProject (79) : : : +- CometScan parquet spark_catalog.default.store_sales (22) : : +- ReusedExchange (28) : +- BroadcastExchange (56) - : +- * ColumnarToRow (55) - : +- CometFilter (54) - : +- CometHashAggregate (53) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- * ColumnarToRow (53) : +- CometColumnarExchange (52) : +- RowToColumnar (51) : +- * HashAggregate (50) @@ -61,8 +61,8 @@ TakeOrderedAndProject (79) : : +- CometScan parquet spark_catalog.default.web_sales (41) : +- ReusedExchange (47) +- BroadcastExchange (76) - +- * ColumnarToRow (75) - +- CometHashAggregate (74) + +- * HashAggregate (75) + +- * ColumnarToRow (74) +- CometColumnarExchange (73) +- RowToColumnar (72) +- * HashAggregate (71) @@ -150,297 +150,305 @@ Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 16] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] + +(17) HashAggregate [codegen id : 16] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] +Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] -(17) CometFilter -Input [2]: [customer_id#13, year_total#14] -Condition : (isnotnull(year_total#14) AND (year_total#14 > 0.00)) - -(18) ColumnarToRow [codegen id : 16] -Input [2]: [customer_id#13, year_total#14] +(18) Filter [codegen id : 16] +Input [2]: [customer_id#14, year_total#15] +Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] +Output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (20) CometFilter -Input [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_customer_id#16)) +Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] +Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (23) CometFilter -Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_customer_sk#19) +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#20) (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] (25) BroadcastExchange -Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#15] -Right keys [1]: [ss_customer_sk#19] +Left keys [1]: [c_customer_sk#16] +Right keys [1]: [ss_customer_sk#20] Join type: Inner Join condition: None (27) Project [codegen id : 6] -Output [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21] -Input [7]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18, ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] +Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] (28) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#23, d_year#24] +Output [2]: [d_date_sk#24, d_year#25] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#21] -Right keys [1]: [d_date_sk#23] +Left keys [1]: [ss_sold_date_sk#22] +Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None (30) Project [codegen id : 6] -Output [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] -Input [7]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21, d_date_sk#23, d_year#24] +Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] +Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] (31) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] -Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#20))] -Aggregate Attributes [1]: [sum#25] -Results [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] +Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] +Aggregate Attributes [1]: [sum#26] +Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] (32) RowToColumnar -Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] (33) CometColumnarExchange -Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] -Arguments: hashpartitioning(c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] +Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometHashAggregate -Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] -Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] -Functions [1]: [sum(UnscaledValue(ss_net_paid#20))] +(34) ColumnarToRow [codegen id : 7] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -(35) ColumnarToRow [codegen id : 7] -Input [4]: [customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30] +(35) HashAggregate [codegen id : 7] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] +Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] +Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] +Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] (36) BroadcastExchange -Input [4]: [customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30] +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] (37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#13] -Right keys [1]: [customer_id#27] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#28] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] +Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (39) CometFilter -Input [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] -Condition : (isnotnull(c_customer_sk#31) AND isnotnull(c_customer_id#32)) +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) (40) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] +Output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (42) CometFilter -Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] -Condition : isnotnull(ws_bill_customer_sk#35) +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_bill_customer_sk#36) (43) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] (44) BroadcastExchange -Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] (45) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#31] -Right keys [1]: [ws_bill_customer_sk#35] +Left keys [1]: [c_customer_sk#32] +Right keys [1]: [ws_bill_customer_sk#36] Join type: Inner Join condition: None (46) Project [codegen id : 10] -Output [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, ws_sold_date_sk#37] -Input [7]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] +Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] +Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] (47) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#39, d_year#40] +Output [2]: [d_date_sk#40, d_year#41] (48) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None (49) Project [codegen id : 10] -Output [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, d_year#40] -Input [7]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, ws_sold_date_sk#37, d_date_sk#39, d_year#40] +Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] +Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] (50) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, d_year#40] -Keys [4]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#36))] -Aggregate Attributes [1]: [sum#41] -Results [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] +Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] +Aggregate Attributes [1]: [sum#42] +Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] (51) RowToColumnar -Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] (52) CometColumnarExchange -Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] -Arguments: hashpartitioning(c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometHashAggregate -Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] -Keys [4]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40] -Functions [1]: [sum(UnscaledValue(ws_net_paid#36))] +(53) ColumnarToRow [codegen id : 11] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -(54) CometFilter -Input [2]: [customer_id#43, year_total#44] -Condition : (isnotnull(year_total#44) AND (year_total#44 > 0.00)) +(54) HashAggregate [codegen id : 11] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] +Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] +Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] +Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] -(55) ColumnarToRow [codegen id : 11] -Input [2]: [customer_id#43, year_total#44] +(55) Filter [codegen id : 11] +Input [2]: [customer_id#45, year_total#46] +Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) (56) BroadcastExchange -Input [2]: [customer_id#43, year_total#44] +Input [2]: [customer_id#45, year_total#46] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] (57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#13] -Right keys [1]: [customer_id#43] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#45] Join type: Inner Join condition: None (58) Project [codegen id : 16] -Output [7]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, year_total#44] -Input [8]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, customer_id#43, year_total#44] +Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] +Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] +Output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (60) CometFilter -Input [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] -Condition : (isnotnull(c_customer_sk#45) AND isnotnull(c_customer_id#46)) +Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] +Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] +Output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (63) CometFilter -Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_bill_customer_sk#49) +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_bill_customer_sk#51) (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] (65) BroadcastExchange -Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] (66) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#45] -Right keys [1]: [ws_bill_customer_sk#49] +Left keys [1]: [c_customer_sk#47] +Right keys [1]: [ws_bill_customer_sk#51] Join type: Inner Join condition: None (67) Project [codegen id : 14] -Output [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, ws_sold_date_sk#51] -Input [7]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48, ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] +Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] +Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] (68) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#53, d_year#54] +Output [2]: [d_date_sk#55, d_year#56] (69) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#53] +Left keys [1]: [ws_sold_date_sk#53] +Right keys [1]: [d_date_sk#55] Join type: Inner Join condition: None (70) Project [codegen id : 14] -Output [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, d_year#54] -Input [7]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, ws_sold_date_sk#51, d_date_sk#53, d_year#54] +Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] +Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] (71) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, d_year#54] -Keys [4]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#50))] -Aggregate Attributes [1]: [sum#55] -Results [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] +Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] +Aggregate Attributes [1]: [sum#57] +Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] (72) RowToColumnar -Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] (73) CometColumnarExchange -Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] -Arguments: hashpartitioning(c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometHashAggregate -Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] -Keys [4]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54] -Functions [1]: [sum(UnscaledValue(ws_net_paid#50))] +(74) ColumnarToRow [codegen id : 15] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -(75) ColumnarToRow [codegen id : 15] -Input [2]: [customer_id#57, year_total#58] +(75) HashAggregate [codegen id : 15] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] +Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#52))#44] +Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#52))#44,17,2) AS year_total#60] (76) BroadcastExchange -Input [2]: [customer_id#57, year_total#58] +Input [2]: [customer_id#59, year_total#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] (77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#13] -Right keys [1]: [customer_id#57] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#59] Join type: Inner -Join condition: (CASE WHEN (year_total#44 > 0.00) THEN (year_total#58 / year_total#44) END > CASE WHEN (year_total#14 > 0.00) THEN (year_total#30 / year_total#14) END) +Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) (78) Project [codegen id : 16] -Output [3]: [customer_id#27, customer_first_name#28, customer_last_name#29] -Input [9]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, year_total#44, customer_id#57, year_total#58] +Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] (79) TakeOrderedAndProject -Input [3]: [customer_id#27, customer_first_name#28, customer_last_name#29] -Arguments: 100, [customer_id#27 ASC NULLS FIRST, customer_id#27 ASC NULLS FIRST, customer_id#27 ASC NULLS FIRST], [customer_id#27, customer_first_name#28, customer_last_name#29] +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== @@ -469,7 +477,7 @@ Input [2]: [d_date_sk#9, d_year#10] Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (87) +- * ColumnarToRow (86) +- CometFilter (85) @@ -477,25 +485,25 @@ BroadcastExchange (87) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_year#24] +Output [2]: [d_date_sk#24, d_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (85) CometFilter -Input [2]: [d_date_sk#23, d_year#24] -Condition : (((isnotnull(d_year#24) AND (d_year#24 = 2002)) AND d_year#24 IN (2001,2002)) AND isnotnull(d_date_sk#23)) +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) (86) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#23, d_year#24] +Input [2]: [d_date_sk#24, d_year#25] (87) BroadcastExchange -Input [2]: [d_date_sk#23, d_year#24] +Input [2]: [d_date_sk#24, d_year#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#22 +Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 6136429173..1877fb7e14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -5,10 +5,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] - ColumnarToRow - InputAdapter - CometFilter [year_total] - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 RowToColumnar WholeStageCodegen (3) @@ -40,9 +40,9 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 RowToColumnar WholeStageCodegen (6) @@ -74,10 +74,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [year_total] - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 RowToColumnar WholeStageCodegen (10) @@ -103,9 +103,9 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 RowToColumnar WholeStageCodegen (14) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index 3286597a34..55745424ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) +TakeOrderedAndProject (40) ++- * HashAggregate (39) + +- * ColumnarToRow (38) +- CometColumnarExchange (37) +- RowToColumnar (36) +- * HashAggregate (35) @@ -210,15 +210,17 @@ Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(38) CometHashAggregate +(38) ColumnarToRow [codegen id : 11] +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] + +(39) HashAggregate [codegen id : 11] Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] Keys [5]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] +Aggregate Attributes [2]: [count(1)#41, sum(UnscaledValue(ext_sales_price#12))#42] +Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count(1)#41 AS sales_cnt#43, MakeDecimal(sum(UnscaledValue(ext_sales_price#12))#42,17,2) AS sales_amt#44] -(39) CometTakeOrderedAndProject -Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#41, sales_amt#42] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#10 ASC NULLS FIRST,col_name#11 ASC NULLS FIRST,d_year#8 ASC NULLS FIRST,d_qoy#9 ASC NULLS FIRST,i_category#6 ASC NULLS FIRST], output=[channel#10,col_name#11,d_year#8,d_qoy#9,i_category#6,sales_cnt#41,sales_amt#42]), 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#41, sales_amt#42] - -(40) ColumnarToRow [codegen id : 11] -Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#41, sales_amt#42] +(40) TakeOrderedAndProject +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] +Arguments: 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index 8cd2591c7b..0b6532be90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] +TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] + WholeStageCodegen (11) + HashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] [count(1),sum(UnscaledValue(ext_sales_price)),sales_cnt,sales_amt,count,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [channel,col_name,d_year,d_qoy,i_category] #1 RowToColumnar WholeStageCodegen (10) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index 2b95b83f04..66cfac4c6b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (99) -+- CometTakeOrderedAndProject (98) - +- CometHashAggregate (97) +TakeOrderedAndProject (99) ++- * HashAggregate (98) + +- * ColumnarToRow (97) +- CometColumnarExchange (96) +- RowToColumnar (95) +- * HashAggregate (94) @@ -9,8 +9,8 @@ +- Union (92) :- * Project (34) : +- * BroadcastHashJoin LeftOuter BuildRight (33) - : :- * ColumnarToRow (17) - : : +- CometHashAggregate (16) + : :- * HashAggregate (17) + : : +- * ColumnarToRow (16) : : +- CometColumnarExchange (15) : : +- RowToColumnar (14) : : +- * HashAggregate (13) @@ -27,8 +27,8 @@ : : +- CometFilter (8) : : +- CometScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) - : +- CometHashAggregate (30) + : +- * HashAggregate (31) + : +- * ColumnarToRow (30) : +- CometColumnarExchange (29) : +- RowToColumnar (28) : +- * HashAggregate (27) @@ -44,8 +44,8 @@ :- * Project (57) : +- * BroadcastNestedLoopJoin Inner BuildLeft (56) : :- BroadcastExchange (45) - : : +- * ColumnarToRow (44) - : : +- CometHashAggregate (43) + : : +- * HashAggregate (44) + : : +- * ColumnarToRow (43) : : +- CometColumnarExchange (42) : : +- RowToColumnar (41) : : +- * HashAggregate (40) @@ -54,8 +54,8 @@ : : :- * ColumnarToRow (36) : : : +- CometScan parquet spark_catalog.default.catalog_sales (35) : : +- ReusedExchange (37) - : +- * ColumnarToRow (55) - : +- CometHashAggregate (54) + : +- * HashAggregate (55) + : +- * ColumnarToRow (54) : +- CometColumnarExchange (53) : +- RowToColumnar (52) : +- * HashAggregate (51) @@ -66,8 +66,8 @@ : +- ReusedExchange (48) +- * Project (91) +- * BroadcastHashJoin LeftOuter BuildRight (90) - :- * ColumnarToRow (74) - : +- CometHashAggregate (73) + :- * HashAggregate (74) + : +- * ColumnarToRow (73) : +- CometColumnarExchange (72) : +- RowToColumnar (71) : +- * HashAggregate (70) @@ -84,8 +84,8 @@ : +- CometFilter (65) : +- CometScan parquet spark_catalog.default.web_page (64) +- BroadcastExchange (89) - +- * ColumnarToRow (88) - +- CometHashAggregate (87) + +- * HashAggregate (88) + +- * ColumnarToRow (87) +- CometColumnarExchange (86) +- RowToColumnar (85) +- * HashAggregate (84) @@ -170,379 +170,393 @@ Input [3]: [s_store_sk#7, sum#10, sum#11] Input [3]: [s_store_sk#7, sum#10, sum#11] Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 8] +Input [3]: [s_store_sk#7, sum#10, sum#11] + +(17) HashAggregate [codegen id : 8] Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) ColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sales#12, profit#13] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] +Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] (unknown) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] +Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(sr_returned_date_sk#17 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (19) CometFilter -Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] -Condition : isnotnull(sr_store_sk#14) +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#16) (20) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] (21) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#21] (22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#17] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [d_date_sk#21] Join type: Inner Join condition: None (23) Project [codegen id : 6] -Output [3]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16] -Input [5]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17, d_date_sk#19] +Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] +Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#20] +Output [1]: [s_store_sk#22] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#14] -Right keys [1]: [s_store_sk#20] +Left keys [1]: [sr_store_sk#16] +Right keys [1]: [s_store_sk#22] Join type: Inner Join condition: None (26) Project [codegen id : 6] -Output [3]: [sr_return_amt#15, sr_net_loss#16, s_store_sk#20] -Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, s_store_sk#20] +Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] (27) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#15, sr_net_loss#16, s_store_sk#20] -Keys [1]: [s_store_sk#20] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#15)), partial_sum(UnscaledValue(sr_net_loss#16))] -Aggregate Attributes [2]: [sum#21, sum#22] -Results [3]: [s_store_sk#20, sum#23, sum#24] +Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Keys [1]: [s_store_sk#22] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum#23, sum#24] +Results [3]: [s_store_sk#22, sum#25, sum#26] (28) RowToColumnar -Input [3]: [s_store_sk#20, sum#23, sum#24] +Input [3]: [s_store_sk#22, sum#25, sum#26] (29) CometColumnarExchange -Input [3]: [s_store_sk#20, sum#23, sum#24] -Arguments: hashpartitioning(s_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Input [3]: [s_store_sk#22, sum#25, sum#26] +Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometHashAggregate -Input [3]: [s_store_sk#20, sum#23, sum#24] -Keys [1]: [s_store_sk#20] -Functions [2]: [sum(UnscaledValue(sr_return_amt#15)), sum(UnscaledValue(sr_net_loss#16))] +(30) ColumnarToRow [codegen id : 7] +Input [3]: [s_store_sk#22, sum#25, sum#26] -(31) ColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#20, returns#25, profit_loss#26] +(31) HashAggregate [codegen id : 7] +Input [3]: [s_store_sk#22, sum#25, sum#26] +Keys [1]: [s_store_sk#22] +Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] +Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] (32) BroadcastExchange -Input [3]: [s_store_sk#20, returns#25, profit_loss#26] +Input [3]: [s_store_sk#22, returns#29, profit_loss#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#20] +Right keys [1]: [s_store_sk#22] Join type: LeftOuter Join condition: None (34) Project [codegen id : 8] -Output [5]: [sales#12, coalesce(returns#25, 0.00) AS returns#27, (profit#13 - coalesce(profit_loss#26, 0.00)) AS profit#28, store channel AS channel#29, s_store_sk#7 AS id#30] -Input [6]: [s_store_sk#7, sales#12, profit#13, s_store_sk#20, returns#25, profit_loss#26] +Output [5]: [sales#14, coalesce(returns#29, 0.00) AS returns#31, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#32, store channel AS channel#33, s_store_sk#7 AS id#34] +Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] (37) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#36] +Output [1]: [d_date_sk#40] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#34] -Right keys [1]: [d_date_sk#36] +Left keys [1]: [cs_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33] -Input [5]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, d_date_sk#36] +Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33] -Keys [1]: [cs_call_center_sk#31] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(UnscaledValue(cs_net_profit#33))] -Aggregate Attributes [2]: [sum#37, sum#38] -Results [3]: [cs_call_center_sk#31, sum#39, sum#40] +Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum#41, sum#42] +Results [3]: [cs_call_center_sk#35, sum#43, sum#44] (41) RowToColumnar -Input [3]: [cs_call_center_sk#31, sum#39, sum#40] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] (42) CometColumnarExchange -Input [3]: [cs_call_center_sk#31, sum#39, sum#40] -Arguments: hashpartitioning(cs_call_center_sk#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) CometHashAggregate -Input [3]: [cs_call_center_sk#31, sum#39, sum#40] -Keys [1]: [cs_call_center_sk#31] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(UnscaledValue(cs_net_profit#33))] +(43) ColumnarToRow [codegen id : 11] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -(44) ColumnarToRow [codegen id : 11] -Input [3]: [cs_call_center_sk#31, sales#41, profit#42] +(44) HashAggregate [codegen id : 11] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] +Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] (45) BroadcastExchange -Input [3]: [cs_call_center_sk#31, sales#41, profit#42] +Input [3]: [cs_call_center_sk#35, sales#47, profit#48] Arguments: IdentityBroadcastMode, [plan_id=6] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] ReadSchema: struct (47) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] (48) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#47] +Output [1]: [d_date_sk#53] (49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#45] -Right keys [1]: [d_date_sk#47] +Left keys [1]: [cr_returned_date_sk#51] +Right keys [1]: [d_date_sk#53] Join type: Inner Join condition: None (50) Project [codegen id : 13] -Output [2]: [cr_return_amount#43, cr_net_loss#44] -Input [4]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45, d_date_sk#47] +Output [2]: [cr_return_amount#49, cr_net_loss#50] +Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] (51) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#43, cr_net_loss#44] +Input [2]: [cr_return_amount#49, cr_net_loss#50] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#43)), partial_sum(UnscaledValue(cr_net_loss#44))] -Aggregate Attributes [2]: [sum#48, sum#49] -Results [2]: [sum#50, sum#51] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum#54, sum#55] +Results [2]: [sum#56, sum#57] (52) RowToColumnar -Input [2]: [sum#50, sum#51] +Input [2]: [sum#56, sum#57] (53) CometColumnarExchange -Input [2]: [sum#50, sum#51] +Input [2]: [sum#56, sum#57] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(54) CometHashAggregate -Input [2]: [sum#50, sum#51] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#43)), sum(UnscaledValue(cr_net_loss#44))] +(54) ColumnarToRow +Input [2]: [sum#56, sum#57] -(55) ColumnarToRow -Input [2]: [returns#52, profit_loss#53] +(55) HashAggregate +Input [2]: [sum#56, sum#57] +Keys: [] +Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] (56) BroadcastNestedLoopJoin [codegen id : 14] Join type: Inner Join condition: None (57) Project [codegen id : 14] -Output [5]: [sales#41, returns#52, (profit#42 - profit_loss#53) AS profit#54, catalog channel AS channel#55, cs_call_center_sk#31 AS id#56] -Input [5]: [cs_call_center_sk#31, sales#41, profit#42, returns#52, profit_loss#53] +Output [5]: [sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#62, catalog channel AS channel#63, cs_call_center_sk#35 AS id#64] +Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] +Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (59) CometFilter -Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] -Condition : isnotnull(ws_web_page_sk#57) +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +Condition : isnotnull(ws_web_page_sk#65) (60) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] (61) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#62] +Output [1]: [d_date_sk#70] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#60] -Right keys [1]: [d_date_sk#62] +Left keys [1]: [ws_sold_date_sk#68] +Right keys [1]: [d_date_sk#70] Join type: Inner Join condition: None (63) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59] -Input [5]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60, d_date_sk#62] +Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] +Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] (unknown) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#63] +Output [1]: [wp_web_page_sk#71] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (65) CometFilter -Input [1]: [wp_web_page_sk#63] -Condition : isnotnull(wp_web_page_sk#63) +Input [1]: [wp_web_page_sk#71] +Condition : isnotnull(wp_web_page_sk#71) (66) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#63] +Input [1]: [wp_web_page_sk#71] (67) BroadcastExchange -Input [1]: [wp_web_page_sk#63] +Input [1]: [wp_web_page_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] (68) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#57] -Right keys [1]: [wp_web_page_sk#63] +Left keys [1]: [ws_web_page_sk#65] +Right keys [1]: [wp_web_page_sk#71] Join type: Inner Join condition: None (69) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] -Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] +Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] (70) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] -Keys [1]: [wp_web_page_sk#63] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#58)), partial_sum(UnscaledValue(ws_net_profit#59))] -Aggregate Attributes [2]: [sum#64, sum#65] -Results [3]: [wp_web_page_sk#63, sum#66, sum#67] +Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum#72, sum#73] +Results [3]: [wp_web_page_sk#71, sum#74, sum#75] (71) RowToColumnar -Input [3]: [wp_web_page_sk#63, sum#66, sum#67] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] (72) CometColumnarExchange -Input [3]: [wp_web_page_sk#63, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(73) CometHashAggregate -Input [3]: [wp_web_page_sk#63, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#63] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#58)), sum(UnscaledValue(ws_net_profit#59))] +(73) ColumnarToRow [codegen id : 22] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -(74) ColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#63, sales#68, profit#69] +(74) HashAggregate [codegen id : 22] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] +Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] (unknown) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] +Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#73), dynamicpruningexpression(wr_returned_date_sk#73 IN dynamicpruning#74)] +PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (76) CometFilter -Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] -Condition : isnotnull(wr_web_page_sk#70) +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +Condition : isnotnull(wr_web_page_sk#80) (77) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] (78) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#75] +Output [1]: [d_date_sk#85] (79) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#73] -Right keys [1]: [d_date_sk#75] +Left keys [1]: [wr_returned_date_sk#83] +Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None (80) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72] -Input [5]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73, d_date_sk#75] +Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] +Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] (81) ReusedExchange [Reuses operator id: 67] -Output [1]: [wp_web_page_sk#76] +Output [1]: [wp_web_page_sk#86] (82) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#70] -Right keys [1]: [wp_web_page_sk#76] +Left keys [1]: [wr_web_page_sk#80] +Right keys [1]: [wp_web_page_sk#86] Join type: Inner Join condition: None (83) Project [codegen id : 20] -Output [3]: [wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] -Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] +Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] (84) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] -Keys [1]: [wp_web_page_sk#76] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#71)), partial_sum(UnscaledValue(wr_net_loss#72))] -Aggregate Attributes [2]: [sum#77, sum#78] -Results [3]: [wp_web_page_sk#76, sum#79, sum#80] +Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum#87, sum#88] +Results [3]: [wp_web_page_sk#86, sum#89, sum#90] (85) RowToColumnar -Input [3]: [wp_web_page_sk#76, sum#79, sum#80] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] (86) CometColumnarExchange -Input [3]: [wp_web_page_sk#76, sum#79, sum#80] -Arguments: hashpartitioning(wp_web_page_sk#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(87) CometHashAggregate -Input [3]: [wp_web_page_sk#76, sum#79, sum#80] -Keys [1]: [wp_web_page_sk#76] -Functions [2]: [sum(UnscaledValue(wr_return_amt#71)), sum(UnscaledValue(wr_net_loss#72))] +(87) ColumnarToRow [codegen id : 21] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -(88) ColumnarToRow [codegen id : 21] -Input [3]: [wp_web_page_sk#76, returns#81, profit_loss#82] +(88) HashAggregate [codegen id : 21] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] +Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] (89) BroadcastExchange -Input [3]: [wp_web_page_sk#76, returns#81, profit_loss#82] +Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] (90) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#63] -Right keys [1]: [wp_web_page_sk#76] +Left keys [1]: [wp_web_page_sk#71] +Right keys [1]: [wp_web_page_sk#86] Join type: LeftOuter Join condition: None (91) Project [codegen id : 22] -Output [5]: [sales#68, coalesce(returns#81, 0.00) AS returns#83, (profit#69 - coalesce(profit_loss#82, 0.00)) AS profit#84, web channel AS channel#85, wp_web_page_sk#63 AS id#86] -Input [6]: [wp_web_page_sk#63, sales#68, profit#69, wp_web_page_sk#76, returns#81, profit_loss#82] +Output [5]: [sales#78, coalesce(returns#93, 0.00) AS returns#95, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#96, web channel AS channel#97, wp_web_page_sk#71 AS id#98] +Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] (92) Union (93) Expand [codegen id : 23] -Input [5]: [sales#12, returns#27, profit#28, channel#29, id#30] -Arguments: [[sales#12, returns#27, profit#28, channel#29, id#30, 0], [sales#12, returns#27, profit#28, channel#29, null, 1], [sales#12, returns#27, profit#28, null, null, 3]], [sales#12, returns#27, profit#28, channel#87, id#88, spark_grouping_id#89] +Input [5]: [sales#14, returns#31, profit#32, channel#33, id#34] +Arguments: [[sales#14, returns#31, profit#32, channel#33, id#34, 0], [sales#14, returns#31, profit#32, channel#33, null, 1], [sales#14, returns#31, profit#32, null, null, 3]], [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] (94) HashAggregate [codegen id : 23] -Input [6]: [sales#12, returns#27, profit#28, channel#87, id#88, spark_grouping_id#89] -Keys [3]: [channel#87, id#88, spark_grouping_id#89] -Functions [3]: [partial_sum(sales#12), partial_sum(returns#27), partial_sum(profit#28)] -Aggregate Attributes [6]: [sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Results [9]: [channel#87, id#88, spark_grouping_id#89, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Input [6]: [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] +Keys [3]: [channel#99, id#100, spark_grouping_id#101] +Functions [3]: [partial_sum(sales#14), partial_sum(returns#31), partial_sum(profit#32)] +Aggregate Attributes [6]: [sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] +Results [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] (95) RowToColumnar -Input [9]: [channel#87, id#88, spark_grouping_id#89, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] (96) CometColumnarExchange -Input [9]: [channel#87, id#88, spark_grouping_id#89, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Arguments: hashpartitioning(channel#87, id#88, spark_grouping_id#89, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Arguments: hashpartitioning(channel#99, id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(97) CometHashAggregate -Input [9]: [channel#87, id#88, spark_grouping_id#89, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Keys [3]: [channel#87, id#88, spark_grouping_id#89] -Functions [3]: [sum(sales#12), sum(returns#27), sum(profit#28)] +(97) ColumnarToRow [codegen id : 24] +Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -(98) CometTakeOrderedAndProject -Input [5]: [channel#87, id#88, sales#102, returns#103, profit#104] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#87 ASC NULLS FIRST,id#88 ASC NULLS FIRST], output=[channel#87,id#88,sales#102,returns#103,profit#104]), 100, [channel#87 ASC NULLS FIRST, id#88 ASC NULLS FIRST], [channel#87, id#88, sales#102, returns#103, profit#104] +(98) HashAggregate [codegen id : 24] +Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Keys [3]: [channel#99, id#100, spark_grouping_id#101] +Functions [3]: [sum(sales#14), sum(returns#31), sum(profit#32)] +Aggregate Attributes [3]: [sum(sales#14)#114, sum(returns#31)#115, sum(profit#32)#116] +Results [5]: [channel#99, id#100, sum(sales#14)#114 AS sales#117, sum(returns#31)#115 AS returns#118, sum(profit#32)#116 AS profit#119] -(99) ColumnarToRow [codegen id : 24] -Input [5]: [channel#87, id#88, sales#102, returns#103, profit#104] +(99) TakeOrderedAndProject +Input [5]: [channel#99, id#100, sales#117, returns#118, profit#119] +Arguments: 100, [channel#99 ASC NULLS FIRST, id#100 ASC NULLS FIRST], [channel#99, id#100, sales#117, returns#118, profit#119] ===== Subqueries ===== @@ -555,18 +569,18 @@ BroadcastExchange (104) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#105] +Output [2]: [d_date_sk#6, d_date#120] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] ReadSchema: struct (101) CometFilter -Input [2]: [d_date_sk#6, d_date#105] -Condition : (((isnotnull(d_date#105) AND (d_date#105 >= 2000-08-03)) AND (d_date#105 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_date#120] +Condition : (((isnotnull(d_date#120) AND (d_date#120 >= 2000-08-03)) AND (d_date#120 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) (102) CometProject -Input [2]: [d_date_sk#6, d_date#105] +Input [2]: [d_date_sk#6, d_date#120] Arguments: [d_date_sk#6], [d_date_sk#6] (103) ColumnarToRow [codegen id : 1] @@ -576,14 +590,14 @@ Input [1]: [d_date_sk#6] Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 46 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 46 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 75 Hosting Expression = wr_returned_date_sk#73 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 75 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index d8ada5be65..f0ab076bf8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (24) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (24) + 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] + ColumnarToRow + InputAdapter CometColumnarExchange [channel,id,spark_grouping_id] #1 RowToColumnar WholeStageCodegen (23) @@ -13,9 +13,9 @@ WholeStageCodegen (24) WholeStageCodegen (8) Project [sales,returns,profit,profit_loss,s_store_sk] BroadcastHashJoin [s_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometHashAggregate [s_store_sk,sum,sum] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [s_store_sk] #2 RowToColumnar WholeStageCodegen (3) @@ -48,9 +48,9 @@ WholeStageCodegen (24) InputAdapter BroadcastExchange #5 WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [s_store_sk,sum,sum] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [s_store_sk] #6 RowToColumnar WholeStageCodegen (6) @@ -74,9 +74,9 @@ WholeStageCodegen (24) InputAdapter BroadcastExchange #7 WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_call_center_sk,sum,sum] + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [cs_call_center_sk] #8 RowToColumnar WholeStageCodegen (10) @@ -89,9 +89,9 @@ WholeStageCodegen (24) ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - ColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange #9 RowToColumnar WholeStageCodegen (13) @@ -107,9 +107,9 @@ WholeStageCodegen (24) WholeStageCodegen (22) Project [sales,returns,profit,profit_loss,wp_web_page_sk] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - ColumnarToRow - InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [wp_web_page_sk] #10 RowToColumnar WholeStageCodegen (17) @@ -135,9 +135,9 @@ WholeStageCodegen (24) InputAdapter BroadcastExchange #12 WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [wp_web_page_sk] #13 RowToColumnar WholeStageCodegen (20) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index 79c2b2a5bd..f633020bc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -1,75 +1,77 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * ColumnarToRow (69) - +- CometSortMergeJoin (68) - :- CometProject (45) - : +- CometSortMergeJoin (44) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometColumnarExchange (19) - : : +- RowToColumnar (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometColumnarExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometColumnarExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (14) - : +- CometSort (43) - : +- CometFilter (42) - : +- CometHashAggregate (41) - : +- CometColumnarExchange (40) - : +- RowToColumnar (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * ColumnarToRow (34) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometColumnarExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (35) - +- CometSort (67) - +- CometFilter (66) - +- CometHashAggregate (65) - +- CometColumnarExchange (64) - +- RowToColumnar (63) - +- * HashAggregate (62) - +- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * ColumnarToRow (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometColumnarExchange (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (54) - : +- CometColumnarExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan parquet spark_catalog.default.catalog_returns (50) - +- ReusedExchange (59) +TakeOrderedAndProject (73) ++- * Project (72) + +- * SortMergeJoin Inner (71) + :- * Project (47) + : +- * SortMergeJoin Inner (46) + : :- * Sort (22) + : : +- * HashAggregate (21) + : : +- * ColumnarToRow (20) + : : +- CometColumnarExchange (19) + : : +- RowToColumnar (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * ColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : +- ReusedExchange (14) + : +- * Sort (45) + : +- * Filter (44) + : +- * HashAggregate (43) + : +- * ColumnarToRow (42) + : +- CometColumnarExchange (41) + : +- RowToColumnar (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * ColumnarToRow (35) + : : +- CometProject (34) + : : +- CometFilter (33) + : : +- CometSortMergeJoin (32) + : : :- CometSort (26) + : : : +- CometColumnarExchange (25) + : : : +- CometFilter (24) + : : : +- CometScan parquet spark_catalog.default.web_sales (23) + : : +- CometSort (31) + : : +- CometColumnarExchange (30) + : : +- CometProject (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.web_returns (27) + : +- ReusedExchange (36) + +- * Sort (70) + +- * Filter (69) + +- * HashAggregate (68) + +- * ColumnarToRow (67) + +- CometColumnarExchange (66) + +- RowToColumnar (65) + +- * HashAggregate (64) + +- * Project (63) + +- * BroadcastHashJoin Inner BuildRight (62) + :- * ColumnarToRow (60) + : +- CometProject (59) + : +- CometFilter (58) + : +- CometSortMergeJoin (57) + : :- CometSort (51) + : : +- CometColumnarExchange (50) + : : +- CometFilter (49) + : : +- CometScan parquet spark_catalog.default.catalog_sales (48) + : +- CometSort (56) + : +- CometColumnarExchange (55) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan parquet spark_catalog.default.catalog_returns (52) + +- ReusedExchange (61) (unknown) Scan parquet spark_catalog.default.store_sales @@ -131,7 +133,7 @@ Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, (13) ColumnarToRow [codegen id : 2] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(14) ReusedExchange [Reuses operator id: 75] +(14) ReusedExchange [Reuses operator id: 77] Output [2]: [d_date_sk#12, d_year#13] (15) BroadcastHashJoin [codegen id : 2] @@ -158,245 +160,259 @@ Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) CometHashAggregate +(20) ColumnarToRow [codegen id : 3] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] + +(21) HashAggregate [codegen id : 3] Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] +Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) CometSort -Input [6]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23], [ss_sold_year#20 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] +(22) Sort [codegen id : 3] +Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(23) CometFilter -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Condition : (isnotnull(ws_item_sk#24) AND isnotnull(ws_bill_customer_sk#25)) +(24) CometFilter +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) -(24) CometColumnarExchange -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: hashpartitioning(ws_order_number#26, ws_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(25) CometColumnarExchange +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) CometSort -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_order_number#26 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST] +(26) CometSort +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] +Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(27) CometFilter -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Condition : (isnotnull(wr_order_number#33) AND isnotnull(wr_item_sk#32)) +(28) CometFilter +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) -(28) CometProject -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_item_sk#32, wr_order_number#33] +(29) CometProject +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] -(29) CometColumnarExchange -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: hashpartitioning(wr_order_number#33, wr_item_sk#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(30) CometColumnarExchange +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(30) CometSort -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_order_number#33 ASC NULLS FIRST, wr_item_sk#32 ASC NULLS FIRST] +(31) CometSort +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Right output [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [ws_order_number#26, ws_item_sk#24], [wr_order_number#33, wr_item_sk#32], LeftOuter +(32) CometSortMergeJoin +Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Right output [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter -(32) CometFilter -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Condition : isnull(wr_order_number#33) +(33) CometFilter +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Condition : isnull(wr_order_number#36) -(33) CometProject -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +(34) CometProject +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -(34) ColumnarToRow [codegen id : 4] -Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +(35) ColumnarToRow [codegen id : 5] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -(35) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#35, d_year#36] +(36) ReusedExchange [Reuses operator id: 77] +Output [2]: [d_date_sk#38, d_year#39] -(36) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#35] +(37) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(37) Project [codegen id : 4] -Output [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] -Input [8]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, d_date_sk#35, d_year#36] - -(38) HashAggregate [codegen id : 4] -Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [partial_sum(ws_quantity#27), partial_sum(UnscaledValue(ws_wholesale_cost#28)), partial_sum(UnscaledValue(ws_sales_price#29))] -Aggregate Attributes [3]: [sum#37, sum#38, sum#39] -Results [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] - -(39) RowToColumnar -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] - -(40) CometColumnarExchange -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] -Arguments: hashpartitioning(d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(41) CometHashAggregate -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [sum(ws_quantity#27), sum(UnscaledValue(ws_wholesale_cost#28)), sum(UnscaledValue(ws_sales_price#29))] - -(42) CometFilter -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Condition : (coalesce(ws_qty#45, 0) > 0) - -(43) CometSort -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Arguments: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47], [ws_sold_year#43 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST, ws_customer_sk#44 ASC NULLS FIRST] - -(44) CometSortMergeJoin -Left output [6]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Right output [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44], Inner +(38) Project [codegen id : 5] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] + +(39) HashAggregate [codegen id : 5] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum#40, sum#41, sum#42] +Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(40) RowToColumnar +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(41) CometColumnarExchange +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(42) ColumnarToRow [codegen id : 6] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(43) HashAggregate [codegen id : 6] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] +Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] + +(44) Filter [codegen id : 6] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Condition : (coalesce(ws_qty#51, 0) > 0) + +(45) Sort [codegen id : 6] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 + +(46) SortMergeJoin [codegen id : 7] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] +Join type: Inner +Join condition: None -(45) CometProject -Input [12]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47], [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] +(47) Project [codegen id : 7] +Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] +Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#54), dynamicpruningexpression(cs_sold_date_sk#54 IN dynamicpruning#55)] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter -Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Condition : (isnotnull(cs_item_sk#49) AND isnotnull(cs_bill_customer_sk#48)) +(49) CometFilter +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) -(48) CometColumnarExchange -Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: hashpartitioning(cs_order_number#50, cs_item_sk#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(50) CometColumnarExchange +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(49) CometSort -Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_order_number#50 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST] +(51) CometSort +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] +Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(51) CometFilter -Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] -Condition : (isnotnull(cr_order_number#57) AND isnotnull(cr_item_sk#56)) +(53) CometFilter +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) -(52) CometProject -Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_item_sk#56, cr_order_number#57] +(54) CometProject +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] -(53) CometColumnarExchange -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: hashpartitioning(cr_order_number#57, cr_item_sk#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +(55) CometColumnarExchange +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(54) CometSort -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_order_number#57 ASC NULLS FIRST, cr_item_sk#56 ASC NULLS FIRST] +(56) CometSort +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] -(55) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Right output [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cs_order_number#50, cs_item_sk#49], [cr_order_number#57, cr_item_sk#56], LeftOuter +(57) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Right output [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter -(56) CometFilter -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Condition : isnull(cr_order_number#57) +(58) CometFilter +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Condition : isnull(cr_order_number#63) -(57) CometProject -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(59) CometProject +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(58) ColumnarToRow [codegen id : 6] -Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(60) ColumnarToRow [codegen id : 9] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(59) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#59, d_year#60] +(61) ReusedExchange [Reuses operator id: 77] +Output [2]: [d_date_sk#65, d_year#66] -(60) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#54] -Right keys [1]: [d_date_sk#59] +(62) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_sold_date_sk#60] +Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(61) Project [codegen id : 6] -Output [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] -Input [8]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, d_date_sk#59, d_year#60] - -(62) HashAggregate [codegen id : 6] -Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [partial_sum(cs_quantity#51), partial_sum(UnscaledValue(cs_wholesale_cost#52)), partial_sum(UnscaledValue(cs_sales_price#53))] -Aggregate Attributes [3]: [sum#61, sum#62, sum#63] -Results [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] - -(63) RowToColumnar -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] - -(64) CometColumnarExchange -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] -Arguments: hashpartitioning(d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(65) CometHashAggregate -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [sum(cs_quantity#51), sum(UnscaledValue(cs_wholesale_cost#52)), sum(UnscaledValue(cs_sales_price#53))] - -(66) CometFilter -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Condition : (coalesce(cs_qty#69, 0) > 0) - -(67) CometSort -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Arguments: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71], [cs_sold_year#67 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST, cs_customer_sk#68 ASC NULLS FIRST] - -(68) CometSortMergeJoin -Left output [9]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] -Right output [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68], Inner - -(69) ColumnarToRow [codegen id : 7] -Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] +(63) Project [codegen id : 9] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] + +(64) HashAggregate [codegen id : 9] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum#67, sum#68, sum#69] +Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] + +(65) RowToColumnar +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] + +(66) CometColumnarExchange +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(67) ColumnarToRow [codegen id : 10] +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] + +(68) HashAggregate [codegen id : 10] +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] +Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] + +(69) Filter [codegen id : 10] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Condition : (coalesce(cs_qty#78, 0) > 0) + +(70) Sort [codegen id : 10] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 + +(71) SortMergeJoin [codegen id : 11] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] +Join type: Inner +Join condition: None -(70) Project [codegen id : 7] -Output [12]: [round((cast(ss_qty#21 as double) / cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)), 2) AS ratio#72, ss_qty#21 AS store_qty#73, ss_wc#22 AS store_wholesale_cost#74, ss_sp#23 AS store_sales_price#75, (coalesce(ws_qty#45, 0) + coalesce(cs_qty#69, 0)) AS other_chan_qty#76, (coalesce(ws_wc#46, 0.00) + coalesce(cs_wc#70, 0.00)) AS other_chan_wholesale_cost#77, (coalesce(ws_sp#47, 0.00) + coalesce(cs_sp#71, 0.00)) AS other_chan_sales_price#78, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, cs_qty#69] -Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] +(72) Project [codegen id : 11] +Output [12]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] +Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -(71) TakeOrderedAndProject -Input [12]: [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, cs_qty#69] -Arguments: 100, [ratio#72 ASC NULLS FIRST, ss_qty#21 DESC NULLS LAST, ss_wc#22 DESC NULLS LAST, ss_sp#23 DESC NULLS LAST, other_chan_qty#76 ASC NULLS FIRST, other_chan_wholesale_cost#77 ASC NULLS FIRST, other_chan_sales_price#78 ASC NULLS FIRST, round((cast(ss_qty#21 as double) / cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)), 2) ASC NULLS FIRST], [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78] +(73) TakeOrderedAndProject +Input [12]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] +Arguments: 100, [ratio#81 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (77) ++- * ColumnarToRow (76) + +- CometFilter (75) + +- CometScan parquet spark_catalog.default.date_dim (74) (unknown) Scan parquet spark_catalog.default.date_dim @@ -406,19 +422,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(75) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(74) ColumnarToRow [codegen id : 1] +(76) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(75) BroadcastExchange +(77) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#54 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 48 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index 002e6d86d8..cc0372b8ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -1,91 +1,103 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (7) + WholeStageCodegen (11) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - ColumnarToRow + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk] - CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [sr_ticket_number] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk] - CometFilter [ws_qty] - CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + WholeStageCodegen (7) + Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_sold_year,ss_item_sk,ss_customer_sk] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [sr_ticket_number] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_ticket_number,sr_item_sk] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (6) + Sort [ws_sold_year,ws_item_sk,ws_customer_sk] + Filter [ws_qty] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [wr_order_number] + CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometSort [ws_order_number,ws_item_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #6 + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_order_number,wr_item_sk] + CometColumnarExchange [wr_order_number,wr_item_sk] #7 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (10) + Sort [cs_sold_year,cs_item_sk,cs_customer_sk] + Filter [cs_qty] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 RowToColumnar - WholeStageCodegen (4) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + WholeStageCodegen (9) + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [wr_order_number] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #6 - CometFilter [ws_item_sk,ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cr_order_number] + CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometSort [cs_order_number,cs_item_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #9 + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometSort [wr_order_number,wr_item_sk] - CometColumnarExchange [wr_order_number,wr_item_sk] #7 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometSort [cr_order_number,cr_item_sk] + CometColumnarExchange [cr_order_number,cr_item_sk] #10 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk] - CometFilter [cs_qty] - CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cr_order_number] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #9 - CometFilter [cs_item_sk,cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_order_number,cr_item_sk] - CometColumnarExchange [cr_order_number,cr_item_sk] #10 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt index 344150578b..9bb7383bf2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt @@ -2,8 +2,8 @@ TakeOrderedAndProject (32) +- * Project (31) +- * BroadcastHashJoin Inner BuildRight (30) - :- * ColumnarToRow (25) - : +- CometHashAggregate (24) + :- * HashAggregate (25) + : +- * ColumnarToRow (24) : +- CometColumnarExchange (23) : +- RowToColumnar (22) : +- * HashAggregate (21) @@ -139,45 +139,47 @@ Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#1 Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometHashAggregate +(24) ColumnarToRow [codegen id : 6] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] + +(25) HashAggregate [codegen id : 6] Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(25) ColumnarToRow [codegen id : 6] -Input [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#21, profit#22] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#21, sum(UnscaledValue(ss_net_profit#7))#22] +Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#21,17,2) AS amt#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#22,17,2) AS profit#24] (unknown) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] +Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (27) CometFilter -Input [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] -Condition : isnotnull(c_customer_sk#23) +Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +Condition : isnotnull(c_customer_sk#25) (28) ColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] +Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] (29) BroadcastExchange -Input [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] +Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] (30) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None (31) Project [codegen id : 6] -Output [7]: [c_last_name#25, c_first_name#24, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#21, profit#22, s_city#13] -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#21, profit#22, c_customer_sk#23, c_first_name#24, c_last_name#25] +Output [7]: [c_last_name#27, c_first_name#26, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] +Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#26, c_last_name#27] (32) TakeOrderedAndProject -Input [7]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#21, profit#22, s_city#13] -Arguments: 100, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#22 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#26, ss_ticket_number#5, amt#21, profit#22] +Input [7]: [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] +Arguments: 100, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24] ===== Subqueries ===== @@ -190,18 +192,18 @@ BroadcastExchange (37) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#27, d_dow#28] +Output [3]: [d_date_sk#10, d_year#29, d_dow#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct (34) CometFilter -Input [3]: [d_date_sk#10, d_year#27, d_dow#28] -Condition : (((isnotnull(d_dow#28) AND (d_dow#28 = 1)) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#10, d_year#29, d_dow#30] +Condition : (((isnotnull(d_dow#30) AND (d_dow#30 = 1)) AND d_year#29 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) (35) CometProject -Input [3]: [d_date_sk#10, d_year#27, d_dow#28] +Input [3]: [d_date_sk#10, d_year#29, d_dow#30] Arguments: [d_date_sk#10], [d_date_sk#10] (36) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt index b86117fc50..e4fa69b477 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt @@ -2,9 +2,9 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, WholeStageCodegen (6) Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 RowToColumnar WholeStageCodegen (4) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index 67656a9098..621905e5af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (49) -+- CometTakeOrderedAndProject (48) - +- CometHashAggregate (47) +TakeOrderedAndProject (49) ++- * HashAggregate (48) + +- * ColumnarToRow (47) +- CometColumnarExchange (46) +- RowToColumnar (45) +- * HashAggregate (44) @@ -31,10 +31,10 @@ : +- CometFilter (14) : +- CometScan parquet spark_catalog.default.customer_address (13) +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometHashAggregate (30) + +- * Project (33) + +- * Filter (32) + +- * HashAggregate (31) + +- * ColumnarToRow (30) +- CometColumnarExchange (29) +- RowToColumnar (28) +- * HashAggregate (27) @@ -183,29 +183,31 @@ Input [2]: [ca_zip#12, count#16] Input [2]: [ca_zip#12, count#16] Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometHashAggregate +(30) ColumnarToRow [codegen id : 5] +Input [2]: [ca_zip#12, count#16] + +(31) HashAggregate [codegen id : 5] Input [2]: [ca_zip#12, count#16] Keys [1]: [ca_zip#12] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#17] +Results [2]: [substr(ca_zip#12, 1, 5) AS ca_zip#18, count(1)#17 AS cnt#19] -(31) CometFilter -Input [2]: [ca_zip#17, cnt#18] -Condition : (cnt#18 > 10) - -(32) CometProject -Input [2]: [ca_zip#17, cnt#18] -Arguments: [ca_zip#17], [ca_zip#17] +(32) Filter [codegen id : 5] +Input [2]: [ca_zip#18, cnt#19] +Condition : (cnt#19 > 10) -(33) ColumnarToRow [codegen id : 5] -Input [1]: [ca_zip#17] +(33) Project [codegen id : 5] +Output [1]: [ca_zip#18] +Input [2]: [ca_zip#18, cnt#19] (34) BroadcastExchange -Input [1]: [ca_zip#17] +Input [1]: [ca_zip#18] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [plan_id=4] (35) BroadcastHashJoin [codegen id : 6] Left keys [2]: [coalesce(ca_zip#10, ), isnull(ca_zip#10)] -Right keys [2]: [coalesce(ca_zip#17, ), isnull(ca_zip#17)] +Right keys [2]: [coalesce(ca_zip#18, ), isnull(ca_zip#18)] Join type: LeftSemi Join condition: None @@ -249,27 +251,29 @@ Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#10] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#19] -Results [2]: [s_store_name#7, sum#20] +Aggregate Attributes [1]: [sum#20] +Results [2]: [s_store_name#7, sum#21] (45) RowToColumnar -Input [2]: [s_store_name#7, sum#20] +Input [2]: [s_store_name#7, sum#21] (46) CometColumnarExchange -Input [2]: [s_store_name#7, sum#20] +Input [2]: [s_store_name#7, sum#21] Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(47) CometHashAggregate -Input [2]: [s_store_name#7, sum#20] +(47) ColumnarToRow [codegen id : 9] +Input [2]: [s_store_name#7, sum#21] + +(48) HashAggregate [codegen id : 9] +Input [2]: [s_store_name#7, sum#21] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] +Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS sum(ss_net_profit)#23] -(48) CometTakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#7 ASC NULLS FIRST], output=[s_store_name#7,sum(ss_net_profit)#21]), 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#21] - -(49) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_name#7, sum(ss_net_profit)#21] +(49) TakeOrderedAndProject +Input [2]: [s_store_name#7, sum(ss_net_profit)#23] +Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#23] ===== Subqueries ===== @@ -282,18 +286,18 @@ BroadcastExchange (54) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#22, d_qoy#23] +Output [3]: [d_date_sk#5, d_year#24, d_qoy#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct (51) CometFilter -Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] -Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 1998)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] +Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2)) AND (d_year#24 = 1998)) AND isnotnull(d_date_sk#5)) (52) CometProject -Input [3]: [d_date_sk#5, d_year#22, d_qoy#23] +Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] Arguments: [d_date_sk#5], [d_date_sk#5] (53) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index f4b5da283b..df2f062eff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - CometHashAggregate [s_store_name,sum] +TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] + WholeStageCodegen (9) + HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] + ColumnarToRow + InputAdapter CometColumnarExchange [s_store_name] #1 RowToColumnar WholeStageCodegen (8) @@ -53,11 +53,11 @@ WholeStageCodegen (9) InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [ca_zip] - CometFilter [cnt] - CometHashAggregate [ca_zip,count] + Project [ca_zip] + Filter [cnt] + HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_zip] #7 RowToColumnar WholeStageCodegen (4) 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 e08d527aa0..9c098dccbe 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,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (112) -+- CometTakeOrderedAndProject (111) - +- CometHashAggregate (110) +TakeOrderedAndProject (112) ++- * HashAggregate (111) + +- * ColumnarToRow (110) +- CometColumnarExchange (109) +- RowToColumnar (108) +- * HashAggregate (107) @@ -609,17 +609,19 @@ Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, su Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] Arguments: hashpartitioning(channel#111, id#112, spark_grouping_id#113, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(110) CometHashAggregate +(110) ColumnarToRow [codegen id : 20] +Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(111) HashAggregate [codegen id : 20] Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] Keys [3]: [channel#111, id#112, spark_grouping_id#113] Functions [3]: [sum(sales#34), sum(returns#35), sum(profit#36)] +Aggregate Attributes [3]: [sum(sales#34)#126, sum(returns#35)#127, sum(profit#36)#128] +Results [5]: [channel#111, id#112, sum(sales#34)#126 AS sales#129, sum(returns#35)#127 AS returns#130, sum(profit#36)#128 AS profit#131] -(111) CometTakeOrderedAndProject -Input [5]: [channel#111, id#112, sales#126, returns#127, profit#128] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#111 ASC NULLS FIRST,id#112 ASC NULLS FIRST], output=[channel#111,id#112,sales#126,returns#127,profit#128]), 100, [channel#111 ASC NULLS FIRST, id#112 ASC NULLS FIRST], [channel#111, id#112, sales#126, returns#127, profit#128] - -(112) ColumnarToRow [codegen id : 20] -Input [5]: [channel#111, id#112, sales#126, returns#127, profit#128] +(112) TakeOrderedAndProject +Input [5]: [channel#111, id#112, sales#129, returns#130, profit#131] +Arguments: 100, [channel#111 ASC NULLS FIRST, id#112 ASC NULLS FIRST], [channel#111, id#112, sales#129, returns#130, profit#131] ===== Subqueries ===== @@ -632,18 +634,18 @@ BroadcastExchange (117) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#129] +Output [2]: [d_date_sk#14, d_date#132] 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 (114) CometFilter -Input [2]: [d_date_sk#14, d_date#129] -Condition : (((isnotnull(d_date#129) AND (d_date#129 >= 2000-08-23)) AND (d_date#129 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_date#132] +Condition : (((isnotnull(d_date#132) AND (d_date#132 >= 2000-08-23)) AND (d_date#132 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) (115) CometProject -Input [2]: [d_date_sk#14, d_date#129] +Input [2]: [d_date_sk#14, d_date#132] Arguments: [d_date_sk#14], [d_date_sk#14] (116) ColumnarToRow [codegen id : 1] 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 f28c862efd..41a1330584 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,8 +1,8 @@ -WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (20) + 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] + ColumnarToRow + InputAdapter CometColumnarExchange [channel,id,spark_grouping_id] #1 RowToColumnar WholeStageCodegen (19) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index c31369315e..4b9f725b9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * Project (51) - +- * BroadcastHashJoin Inner BuildRight (50) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometHashAggregate (16) +TakeOrderedAndProject (54) ++- * Project (53) + +- * BroadcastHashJoin Inner BuildRight (52) + :- * Project (47) + : +- * BroadcastHashJoin Inner BuildRight (46) + : :- * Project (41) + : : +- * BroadcastHashJoin Inner BuildRight (40) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * ColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- RowToColumnar (14) : : : +- * HashAggregate (13) @@ -24,33 +24,35 @@ TakeOrderedAndProject (52) : : : +- * ColumnarToRow (9) : : : +- CometFilter (8) : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (37) - : : +- * ColumnarToRow (36) - : : +- CometFilter (35) - : : +- CometHashAggregate (34) - : : +- CometColumnarExchange (33) - : : +- CometHashAggregate (32) - : : +- CometHashAggregate (31) - : : +- CometColumnarExchange (30) - : : +- RowToColumnar (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.catalog_returns (19) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (43) - : +- * ColumnarToRow (42) - : +- CometFilter (41) - : +- CometScan parquet spark_catalog.default.customer (40) - +- BroadcastExchange (49) - +- * ColumnarToRow (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.customer_address (46) + : : +- BroadcastExchange (39) + : : +- * Filter (38) + : : +- * HashAggregate (37) + : : +- * ColumnarToRow (36) + : : +- CometColumnarExchange (35) + : : +- RowToColumnar (34) + : : +- * HashAggregate (33) + : : +- * HashAggregate (32) + : : +- * ColumnarToRow (31) + : : +- CometColumnarExchange (30) + : : +- RowToColumnar (29) + : : +- * HashAggregate (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (24) + : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : :- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (19) + : : : +- ReusedExchange (22) + : : +- ReusedExchange (25) + : +- BroadcastExchange (45) + : +- * ColumnarToRow (44) + : +- CometFilter (43) + : +- CometScan parquet spark_catalog.default.customer (42) + +- BroadcastExchange (51) + +- * ColumnarToRow (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.customer_address (48) (unknown) Scan parquet spark_catalog.default.catalog_returns @@ -68,7 +70,7 @@ Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_custom (3) ColumnarToRow [codegen id : 3] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -(4) ReusedExchange [Reuses operator id: 57] +(4) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -123,23 +125,25 @@ Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 11] +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] + +(17) HashAggregate [codegen id : 11] Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] Keys [2]: [cr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#11] +Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] -(17) CometFilter -Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] -Condition : isnotnull(ctr_total_return#13) - -(18) ColumnarToRow [codegen id : 10] -Input [3]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13] +(18) Filter [codegen id : 11] +Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Condition : isnotnull(ctr_total_return#14) (unknown) Scan parquet spark_catalog.default.catalog_returns Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#14)] +PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#15)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct @@ -150,7 +154,7 @@ Condition : isnotnull(cr_returning_addr_sk#2) (21) ColumnarToRow [codegen id : 6] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -(22) ReusedExchange [Reuses operator id: 57] +(22) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#6] (23) BroadcastHashJoin [codegen id : 6] @@ -180,145 +184,157 @@ Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_in Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] Keys [2]: [cr_returning_customer_sk#1, ca_state#8] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#15] -Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] (29) RowToColumnar -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#16] +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] (30) CometColumnarExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#16] +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(31) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#16] +(31) ColumnarToRow [codegen id : 7] +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] + +(32) HashAggregate [codegen id : 7] +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] Keys [2]: [cr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(32) CometHashAggregate -Input [2]: [ctr_state#12, ctr_total_return#13] -Keys [1]: [ctr_state#12] -Functions [1]: [partial_avg(ctr_total_return#13)] - -(33) CometColumnarExchange -Input [3]: [ctr_state#12, sum#17, count#18] -Arguments: hashpartitioning(ctr_state#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [3]: [ctr_state#12, sum#17, count#18] -Keys [1]: [ctr_state#12] -Functions [1]: [avg(ctr_total_return#13)] - -(35) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#19) - -(36) ColumnarToRow [codegen id : 7] -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#11] +Results [2]: [ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] + +(33) HashAggregate [codegen id : 7] +Input [2]: [ctr_state#13, ctr_total_return#14] +Keys [1]: [ctr_state#13] +Functions [1]: [partial_avg(ctr_total_return#14)] +Aggregate Attributes [2]: [sum#18, count#19] +Results [3]: [ctr_state#13, sum#20, count#21] + +(34) RowToColumnar +Input [3]: [ctr_state#13, sum#20, count#21] + +(35) CometColumnarExchange +Input [3]: [ctr_state#13, sum#20, count#21] +Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(36) ColumnarToRow [codegen id : 8] +Input [3]: [ctr_state#13, sum#20, count#21] + +(37) HashAggregate [codegen id : 8] +Input [3]: [ctr_state#13, sum#20, count#21] +Keys [1]: [ctr_state#13] +Functions [1]: [avg(ctr_total_return#14)] +Aggregate Attributes [1]: [avg(ctr_total_return#14)#22] +Results [2]: [(avg(ctr_total_return#14)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#13 AS ctr_state#13#24] + +(38) Filter [codegen id : 8] +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) + +(39) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ctr_state#12] -Right keys [1]: [ctr_state#12#20] +(40) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ctr_state#13] +Right keys [1]: [ctr_state#13#24] Join type: Inner -Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#19) +Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) -(39) Project [codegen id : 10] -Output [2]: [ctr_customer_sk#11, ctr_total_return#13] -Input [5]: [ctr_customer_sk#11, ctr_state#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#19, ctr_state#12#20] +(41) Project [codegen id : 11] +Output [2]: [ctr_customer_sk#12, ctr_total_return#14] +Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] (unknown) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] +Output [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(41) CometFilter -Input [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#23)) +(43) CometFilter +Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] +Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) -(42) ColumnarToRow [codegen id : 8] -Input [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] +(44) ColumnarToRow [codegen id : 9] +Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] -(43) BroadcastExchange -Input [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] +(45) BroadcastExchange +Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ctr_customer_sk#11] -Right keys [1]: [c_customer_sk#21] +(46) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ctr_customer_sk#12] +Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(45) Project [codegen id : 10] -Output [6]: [ctr_total_return#13, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] -Input [8]: [ctr_customer_sk#11, ctr_total_return#13, c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] +(47) Project [codegen id : 11] +Output [6]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] +Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] (unknown) Scan parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#27, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38] +Output [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(47) CometFilter -Input [12]: [ca_address_sk#27, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38] -Condition : ((isnotnull(ca_state#34) AND (ca_state#34 = GA)) AND isnotnull(ca_address_sk#27)) +(49) CometFilter +Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] +Condition : ((isnotnull(ca_state#38) AND (ca_state#38 = GA)) AND isnotnull(ca_address_sk#31)) -(48) ColumnarToRow [codegen id : 9] -Input [12]: [ca_address_sk#27, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38] +(50) ColumnarToRow [codegen id : 10] +Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -(49) BroadcastExchange -Input [12]: [ca_address_sk#27, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38] +(51) BroadcastExchange +Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_current_addr_sk#23] -Right keys [1]: [ca_address_sk#27] +(52) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#27] +Right keys [1]: [ca_address_sk#31] Join type: Inner Join condition: None -(51) Project [codegen id : 10] -Output [16]: [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38, ctr_total_return#13] -Input [18]: [ctr_total_return#13, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, ca_address_sk#27, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38] +(53) Project [codegen id : 11] +Output [16]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] +Input [18]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -(52) TakeOrderedAndProject -Input [16]: [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38, ctr_total_return#13] -Arguments: 100, [c_customer_id#22 ASC NULLS FIRST, c_salutation#24 ASC NULLS FIRST, c_first_name#25 ASC NULLS FIRST, c_last_name#26 ASC NULLS FIRST, ca_street_number#28 ASC NULLS FIRST, ca_street_name#29 ASC NULLS FIRST, ca_street_type#30 ASC NULLS FIRST, ca_suite_number#31 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, ca_state#34 ASC NULLS FIRST, ca_zip#35 ASC NULLS FIRST, ca_country#36 ASC NULLS FIRST, ca_gmt_offset#37 ASC NULLS FIRST, ca_location_type#38 ASC NULLS FIRST, ctr_total_return#13 ASC NULLS FIRST], [c_customer_id#22, c_salutation#24, c_first_name#25, c_last_name#26, ca_street_number#28, ca_street_name#29, ca_street_type#30, ca_suite_number#31, ca_city#32, ca_county#33, ca_state#34, ca_zip#35, ca_country#36, ca_gmt_offset#37, ca_location_type#38, ctr_total_return#13] +(54) TakeOrderedAndProject +Input [16]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] +Arguments: 100, [c_customer_id#26 ASC NULLS FIRST, c_salutation#28 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, c_last_name#30 ASC NULLS FIRST, ca_street_number#32 ASC NULLS FIRST, ca_street_name#33 ASC NULLS FIRST, ca_street_type#34 ASC NULLS FIRST, ca_suite_number#35 ASC NULLS FIRST, ca_city#36 ASC NULLS FIRST, ca_county#37 ASC NULLS FIRST, ca_state#38 ASC NULLS FIRST, ca_zip#39 ASC NULLS FIRST, ca_country#40 ASC NULLS FIRST, ca_gmt_offset#41 ASC NULLS FIRST, ca_location_type#42 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (57) -+- * ColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan parquet spark_catalog.default.date_dim (53) +BroadcastExchange (59) ++- * ColumnarToRow (58) + +- CometProject (57) + +- CometFilter (56) + +- CometScan parquet spark_catalog.default.date_dim (55) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#39] +Output [2]: [d_date_sk#6, d_year#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(54) CometFilter -Input [2]: [d_date_sk#6, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2000)) AND isnotnull(d_date_sk#6)) +(56) CometFilter +Input [2]: [d_date_sk#6, d_year#43] +Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#6)) -(55) CometProject -Input [2]: [d_date_sk#6, d_year#39] +(57) CometProject +Input [2]: [d_date_sk#6, d_year#43] Arguments: [d_date_sk#6], [d_date_sk#6] -(56) ColumnarToRow [codegen id : 1] +(58) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(57) BroadcastExchange +(59) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index 8e5f0eb89b..23c12c9722 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -1,15 +1,15 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - WholeStageCodegen (10) + WholeStageCodegen (11) Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk,ctr_total_return] BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - ColumnarToRow - InputAdapter - CometFilter [ctr_total_return] - CometHashAggregate [cr_returning_customer_sk,ca_state,sum] + Filter [ctr_total_return] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 RowToColumnar WholeStageCodegen (3) @@ -41,41 +41,45 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [(avg(ctr_total_return) * 1.2)] - CometHashAggregate [ctr_state,sum,count] + WholeStageCodegen (8) + Filter [(avg(ctr_total_return) * 1.2)] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ctr_state] #5 - CometHashAggregate [ctr_state,ctr_total_return] - CometHashAggregate [cr_returning_customer_sk,ca_state,sum] - CometColumnarExchange [cr_returning_customer_sk,ca_state] #6 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cr_returning_addr_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [cr_returning_customer_sk,ca_state] #6 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cr_returning_addr_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 - WholeStageCodegen (9) + WholeStageCodegen (10) ColumnarToRow InputAdapter CometFilter [ca_state,ca_address_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 63c12d9f08..22a6f8df6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -4,8 +4,8 @@ TakeOrderedAndProject (52) +- * BroadcastHashJoin Inner BuildRight (50) :- * Project (34) : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * ColumnarToRow (17) - : : +- CometHashAggregate (16) + : :- * HashAggregate (17) + : : +- * ColumnarToRow (16) : : +- CometColumnarExchange (15) : : +- RowToColumnar (14) : : +- * HashAggregate (13) @@ -22,8 +22,8 @@ TakeOrderedAndProject (52) : : : +- CometScan parquet spark_catalog.default.item (4) : : +- ReusedExchange (10) : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) - : +- CometHashAggregate (30) + : +- * HashAggregate (31) + : +- * ColumnarToRow (30) : +- CometColumnarExchange (29) : +- RowToColumnar (28) : +- * HashAggregate (27) @@ -37,8 +37,8 @@ TakeOrderedAndProject (52) : : +- ReusedExchange (21) : +- ReusedExchange (24) +- BroadcastExchange (49) - +- * ColumnarToRow (48) - +- CometHashAggregate (47) + +- * HashAggregate (48) + +- * ColumnarToRow (47) +- CometColumnarExchange (46) +- RowToColumnar (45) +- * HashAggregate (44) @@ -123,171 +123,177 @@ Input [2]: [i_item_id#6, sum#9] Input [2]: [i_item_id#6, sum#9] Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 18] +Input [2]: [i_item_id#6, sum#9] + +(17) HashAggregate [codegen id : 18] Input [2]: [i_item_id#6, sum#9] Keys [1]: [i_item_id#6] Functions [1]: [sum(sr_return_quantity#2)] - -(17) ColumnarToRow [codegen id : 18] -Input [2]: [item_id#10, sr_item_qty#11] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#10] +Results [2]: [i_item_id#6 AS item_id#11, sum(sr_return_quantity#2)#10 AS sr_item_qty#12] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] +Output [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#14), dynamicpruningexpression(cr_returned_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cr_returned_date_sk#15), dynamicpruningexpression(cr_returned_date_sk#15 IN dynamicpruning#16)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct (19) CometFilter -Input [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Condition : isnotnull(cr_item_sk#12) +Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] +Condition : isnotnull(cr_item_sk#13) (20) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] +Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] (21) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#16, i_item_id#17] +Output [2]: [i_item_sk#17, i_item_id#18] (22) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#12] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [cr_item_sk#13] +Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None (23) Project [codegen id : 10] -Output [3]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] -Input [5]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14, i_item_sk#16, i_item_id#17] +Output [3]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18] +Input [5]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15, i_item_sk#17, i_item_id#18] (24) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#19] (25) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#14] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [cr_returned_date_sk#15] +Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None (26) Project [codegen id : 10] -Output [2]: [cr_return_quantity#13, i_item_id#17] -Input [4]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17, d_date_sk#18] +Output [2]: [cr_return_quantity#14, i_item_id#18] +Input [4]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18, d_date_sk#19] (27) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#13, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [1]: [partial_sum(cr_return_quantity#13)] -Aggregate Attributes [1]: [sum#19] -Results [2]: [i_item_id#17, sum#20] +Input [2]: [cr_return_quantity#14, i_item_id#18] +Keys [1]: [i_item_id#18] +Functions [1]: [partial_sum(cr_return_quantity#14)] +Aggregate Attributes [1]: [sum#20] +Results [2]: [i_item_id#18, sum#21] (28) RowToColumnar -Input [2]: [i_item_id#17, sum#20] +Input [2]: [i_item_id#18, sum#21] (29) CometColumnarExchange -Input [2]: [i_item_id#17, sum#20] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Input [2]: [i_item_id#18, sum#21] +Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometHashAggregate -Input [2]: [i_item_id#17, sum#20] -Keys [1]: [i_item_id#17] -Functions [1]: [sum(cr_return_quantity#13)] +(30) ColumnarToRow [codegen id : 11] +Input [2]: [i_item_id#18, sum#21] -(31) ColumnarToRow [codegen id : 11] -Input [2]: [item_id#21, cr_item_qty#22] +(31) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#18, sum#21] +Keys [1]: [i_item_id#18] +Functions [1]: [sum(cr_return_quantity#14)] +Aggregate Attributes [1]: [sum(cr_return_quantity#14)#22] +Results [2]: [i_item_id#18 AS item_id#23, sum(cr_return_quantity#14)#22 AS cr_item_qty#24] (32) BroadcastExchange -Input [2]: [item_id#21, cr_item_qty#22] +Input [2]: [item_id#23, cr_item_qty#24] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] (33) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#10] -Right keys [1]: [item_id#21] +Left keys [1]: [item_id#11] +Right keys [1]: [item_id#23] Join type: Inner Join condition: None (34) Project [codegen id : 18] -Output [3]: [item_id#10, sr_item_qty#11, cr_item_qty#22] -Input [4]: [item_id#10, sr_item_qty#11, item_id#21, cr_item_qty#22] +Output [3]: [item_id#11, sr_item_qty#12, cr_item_qty#24] +Input [4]: [item_id#11, sr_item_qty#12, item_id#23, cr_item_qty#24] (unknown) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] +Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#25), dynamicpruningexpression(wr_returned_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(wr_returned_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct (36) CometFilter -Input [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] -Condition : isnotnull(wr_item_sk#23) +Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] +Condition : isnotnull(wr_item_sk#25) (37) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25] +Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] (38) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#27, i_item_id#28] +Output [2]: [i_item_sk#29, i_item_id#30] (39) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#23] -Right keys [1]: [i_item_sk#27] +Left keys [1]: [wr_item_sk#25] +Right keys [1]: [i_item_sk#29] Join type: Inner Join condition: None (40) Project [codegen id : 16] -Output [3]: [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28] -Input [5]: [wr_item_sk#23, wr_return_quantity#24, wr_returned_date_sk#25, i_item_sk#27, i_item_id#28] +Output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] +Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#29, i_item_id#30] (41) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#31] (42) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#25] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [wr_returned_date_sk#27] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None (43) Project [codegen id : 16] -Output [2]: [wr_return_quantity#24, i_item_id#28] -Input [4]: [wr_return_quantity#24, wr_returned_date_sk#25, i_item_id#28, d_date_sk#29] +Output [2]: [wr_return_quantity#26, i_item_id#30] +Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30, d_date_sk#31] (44) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#24, i_item_id#28] -Keys [1]: [i_item_id#28] -Functions [1]: [partial_sum(wr_return_quantity#24)] -Aggregate Attributes [1]: [sum#30] -Results [2]: [i_item_id#28, sum#31] +Input [2]: [wr_return_quantity#26, i_item_id#30] +Keys [1]: [i_item_id#30] +Functions [1]: [partial_sum(wr_return_quantity#26)] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#30, sum#33] (45) RowToColumnar -Input [2]: [i_item_id#28, sum#31] +Input [2]: [i_item_id#30, sum#33] (46) CometColumnarExchange -Input [2]: [i_item_id#28, sum#31] -Arguments: hashpartitioning(i_item_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [2]: [i_item_id#30, sum#33] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(47) CometHashAggregate -Input [2]: [i_item_id#28, sum#31] -Keys [1]: [i_item_id#28] -Functions [1]: [sum(wr_return_quantity#24)] +(47) ColumnarToRow [codegen id : 17] +Input [2]: [i_item_id#30, sum#33] -(48) ColumnarToRow [codegen id : 17] -Input [2]: [item_id#32, wr_item_qty#33] +(48) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#30, sum#33] +Keys [1]: [i_item_id#30] +Functions [1]: [sum(wr_return_quantity#26)] +Aggregate Attributes [1]: [sum(wr_return_quantity#26)#34] +Results [2]: [i_item_id#30 AS item_id#35, sum(wr_return_quantity#26)#34 AS wr_item_qty#36] (49) BroadcastExchange -Input [2]: [item_id#32, wr_item_qty#33] +Input [2]: [item_id#35, wr_item_qty#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] (50) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#10] -Right keys [1]: [item_id#32] +Left keys [1]: [item_id#11] +Right keys [1]: [item_id#35] Join type: Inner Join condition: None (51) Project [codegen id : 18] -Output [8]: [item_id#10, sr_item_qty#11, (((cast(sr_item_qty#11 as double) / cast(((sr_item_qty#11 + cr_item_qty#22) + wr_item_qty#33) as double)) / 3.0) * 100.0) AS sr_dev#34, cr_item_qty#22, (((cast(cr_item_qty#22 as double) / cast(((sr_item_qty#11 + cr_item_qty#22) + wr_item_qty#33) as double)) / 3.0) * 100.0) AS cr_dev#35, wr_item_qty#33, (((cast(wr_item_qty#33 as double) / cast(((sr_item_qty#11 + cr_item_qty#22) + wr_item_qty#33) as double)) / 3.0) * 100.0) AS wr_dev#36, (cast(((sr_item_qty#11 + cr_item_qty#22) + wr_item_qty#33) as decimal(20,0)) / 3.0) AS average#37] -Input [5]: [item_id#10, sr_item_qty#11, cr_item_qty#22, item_id#32, wr_item_qty#33] +Output [8]: [item_id#11, sr_item_qty#12, (((cast(sr_item_qty#12 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS sr_dev#37, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS cr_dev#38, wr_item_qty#36, (((cast(wr_item_qty#36 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS wr_dev#39, (cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as decimal(20,0)) / 3.0) AS average#40] +Input [5]: [item_id#11, sr_item_qty#12, cr_item_qty#24, item_id#35, wr_item_qty#36] (52) TakeOrderedAndProject -Input [8]: [item_id#10, sr_item_qty#11, sr_dev#34, cr_item_qty#22, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] -Arguments: 100, [item_id#10 ASC NULLS FIRST, sr_item_qty#11 ASC NULLS FIRST], [item_id#10, sr_item_qty#11, sr_dev#34, cr_item_qty#22, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] +Input [8]: [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] +Arguments: 100, [item_id#11 ASC NULLS FIRST, sr_item_qty#12 ASC NULLS FIRST], [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] ===== Subqueries ===== @@ -311,79 +317,79 @@ BroadcastExchange (68) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_date#38] +Output [2]: [d_date_sk#7, d_date#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (54) CometFilter -Input [2]: [d_date_sk#7, d_date#38] +Input [2]: [d_date_sk#7, d_date#41] Condition : isnotnull(d_date_sk#7) (55) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#7, d_date#38] +Input [2]: [d_date_sk#7, d_date#41] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#39, d_week_seq#40] +Output [2]: [d_date#42, d_week_seq#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (57) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#39, d_week_seq#40] +Input [2]: [d_date#42, d_week_seq#43] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] +Output [2]: [d_date#44, d_week_seq#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (59) CometFilter -Input [2]: [d_date#41, d_week_seq#42] -Condition : cast(d_date#41 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Input [2]: [d_date#44, d_week_seq#45] +Condition : cast(d_date#44 as string) IN (2000-06-30,2000-09-27,2000-11-17) (60) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_week_seq#42], [d_week_seq#42] +Input [2]: [d_date#44, d_week_seq#45] +Arguments: [d_week_seq#45], [d_week_seq#45] (61) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#42] +Input [1]: [d_week_seq#45] (62) BroadcastExchange -Input [1]: [d_week_seq#42] +Input [1]: [d_week_seq#45] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] (63) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#40] -Right keys [1]: [d_week_seq#42] +Left keys [1]: [d_week_seq#43] +Right keys [1]: [d_week_seq#45] Join type: LeftSemi Join condition: None (64) Project [codegen id : 2] -Output [1]: [d_date#39] -Input [2]: [d_date#39, d_week_seq#40] +Output [1]: [d_date#42] +Input [2]: [d_date#42, d_week_seq#43] (65) BroadcastExchange -Input [1]: [d_date#39] +Input [1]: [d_date#42] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=8] (66) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#38] -Right keys [1]: [d_date#39] +Left keys [1]: [d_date#41] +Right keys [1]: [d_date#42] Join type: LeftSemi Join condition: None (67) Project [codegen id : 3] Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#38] +Input [2]: [d_date_sk#7, d_date#41] (68) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 18 Hosting Expression = cr_returned_date_sk#14 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 18 Hosting Expression = cr_returned_date_sk#15 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 35 Hosting Expression = wr_returned_date_sk#25 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 35 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index bc1df2c571..1f02d66fef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -4,9 +4,9 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty BroadcastHashJoin [item_id,item_id] Project [item_id,sr_item_qty,cr_item_qty] BroadcastHashJoin [item_id,item_id] - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum] + HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #1 RowToColumnar WholeStageCodegen (5) @@ -56,9 +56,9 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter BroadcastExchange #6 WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum] + HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #7 RowToColumnar WholeStageCodegen (10) @@ -79,9 +79,9 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter BroadcastExchange #8 WholeStageCodegen (17) - ColumnarToRow - InputAdapter - CometHashAggregate [i_item_id,sum] + HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #9 RowToColumnar WholeStageCodegen (16) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index e5c29723a4..2afc9c6c58 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (49) -+- CometTakeOrderedAndProject (48) - +- CometHashAggregate (47) +TakeOrderedAndProject (49) ++- * HashAggregate (48) + +- * ColumnarToRow (47) +- CometColumnarExchange (46) +- RowToColumnar (45) +- * HashAggregate (44) @@ -264,17 +264,19 @@ Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count# Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] Arguments: hashpartitioning(r_reason_desc#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(47) CometHashAggregate +(47) ColumnarToRow [codegen id : 8] +Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] + +(48) HashAggregate [codegen id : 8] Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] Keys [1]: [r_reason_desc#30] Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] +Aggregate Attributes [3]: [avg(ws_quantity#4)#43, avg(UnscaledValue(wr_refunded_cash#16))#44, avg(UnscaledValue(wr_fee#15))#45] +Results [4]: [substr(r_reason_desc#30, 1, 20) AS substr(r_reason_desc, 1, 20)#46, avg(ws_quantity#4)#43 AS avg(ws_quantity)#47, cast((avg(UnscaledValue(wr_refunded_cash#16))#44 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#48, cast((avg(UnscaledValue(wr_fee#15))#45 / 100.0) as decimal(11,6)) AS avg(wr_fee)#49] -(48) CometTakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#43, avg(ws_quantity)#44, avg(wr_refunded_cash)#45, avg(wr_fee)#46] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#43 ASC NULLS FIRST,avg(ws_quantity)#44 ASC NULLS FIRST,avg(wr_refunded_cash)#45 ASC NULLS FIRST,avg(wr_fee)#46 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#43,avg(ws_quantity)#44,avg(wr_refunded_cash)#45,avg(wr_fee)#46]), 100, [substr(r_reason_desc, 1, 20)#43 ASC NULLS FIRST, avg(ws_quantity)#44 ASC NULLS FIRST, avg(wr_refunded_cash)#45 ASC NULLS FIRST, avg(wr_fee)#46 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#43, avg(ws_quantity)#44, avg(wr_refunded_cash)#45, avg(wr_fee)#46] - -(49) ColumnarToRow [codegen id : 8] -Input [4]: [substr(r_reason_desc, 1, 20)#43, avg(ws_quantity)#44, avg(wr_refunded_cash)#45, avg(wr_fee)#46] +(49) TakeOrderedAndProject +Input [4]: [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] +Arguments: 100, [substr(r_reason_desc, 1, 20)#46 ASC NULLS FIRST, avg(ws_quantity)#47 ASC NULLS FIRST, avg(wr_refunded_cash)#48 ASC NULLS FIRST, avg(wr_fee)#49 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] ===== Subqueries ===== @@ -287,18 +289,18 @@ BroadcastExchange (54) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#47] +Output [2]: [d_date_sk#28, d_year#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (51) CometFilter -Input [2]: [d_date_sk#28, d_year#47] -Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2000)) AND isnotnull(d_date_sk#28)) +Input [2]: [d_date_sk#28, d_year#50] +Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2000)) AND isnotnull(d_date_sk#28)) (52) CometProject -Input [2]: [d_date_sk#28, d_year#47] +Input [2]: [d_date_sk#28, d_year#50] Arguments: [d_date_sk#28], [d_date_sk#28] (53) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt index 4d1bf754c5..91d0ae8d09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count] +TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] + WholeStageCodegen (8) + HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [r_reason_desc] #1 RowToColumnar WholeStageCodegen (7) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt index 5bd7bc5594..8e40633a9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt @@ -1,27 +1,29 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * ColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- CometHashAggregate (17) - +- CometColumnarExchange (16) - +- RowToColumnar (15) - +- * HashAggregate (14) - +- * Expand (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (10) - +- * ColumnarToRow (9) - +- CometFilter (8) - +- CometScan parquet spark_catalog.default.item (7) +TakeOrderedAndProject (25) ++- * Project (24) + +- Window (23) + +- * ColumnarToRow (22) + +- CometSort (21) + +- CometColumnarExchange (20) + +- RowToColumnar (19) + +- * HashAggregate (18) + +- * ColumnarToRow (17) + +- CometColumnarExchange (16) + +- RowToColumnar (15) + +- * HashAggregate (14) + +- * Expand (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- ReusedExchange (4) + +- BroadcastExchange (10) + +- * ColumnarToRow (9) + +- CometFilter (8) + +- CometScan parquet spark_catalog.default.item (7) (unknown) Scan parquet spark_catalog.default.web_sales @@ -39,7 +41,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 28] +(4) ReusedExchange [Reuses operator id: 30] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 3] @@ -98,63 +100,71 @@ Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] Arguments: hashpartitioning(i_category#9, i_class#10, spark_grouping_id#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometHashAggregate +(17) ColumnarToRow [codegen id : 4] +Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] + +(18) HashAggregate [codegen id : 4] Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] Keys [3]: [i_category#9, i_class#10, spark_grouping_id#11] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#14] +Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS total_sum#15, i_category#9, i_class#10, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS lochierarchy#16, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS _w0#17, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS _w1#18, CASE WHEN (cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint) = 0) THEN i_category#9 END AS _w2#19] + +(19) RowToColumnar +Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] -(18) CometColumnarExchange -Input [7]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18] -Arguments: hashpartitioning(_w1#17, _w2#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(20) CometColumnarExchange +Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] +Arguments: hashpartitioning(_w1#18, _w2#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(19) CometSort -Input [7]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18] -Arguments: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18], [_w1#17 ASC NULLS FIRST, _w2#18 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] +(21) CometSort +Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] +Arguments: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19], [_w1#18 ASC NULLS FIRST, _w2#19 ASC NULLS FIRST, _w0#17 DESC NULLS LAST] -(20) ColumnarToRow [codegen id : 4] -Input [7]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18] +(22) ColumnarToRow [codegen id : 5] +Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] -(21) Window -Input [7]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18] -Arguments: [rank(_w0#16) windowspecdefinition(_w1#17, _w2#18, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#19], [_w1#17, _w2#18], [_w0#16 DESC NULLS LAST] +(23) Window +Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] +Arguments: [rank(_w0#17) windowspecdefinition(_w1#18, _w2#19, _w0#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#20], [_w1#18, _w2#19], [_w0#17 DESC NULLS LAST] -(22) Project [codegen id : 5] -Output [5]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, rank_within_parent#19] -Input [8]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, _w0#16, _w1#17, _w2#18, rank_within_parent#19] +(24) Project [codegen id : 6] +Output [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] +Input [8]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19, rank_within_parent#20] -(23) TakeOrderedAndProject -Input [5]: [total_sum#14, i_category#9, i_class#10, lochierarchy#15, rank_within_parent#19] -Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#19 ASC NULLS FIRST], [total_sum#14, i_category#9, i_class#10, lochierarchy#15, rank_within_parent#19] +(25) TakeOrderedAndProject +Input [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] +Arguments: 100, [lochierarchy#16 DESC NULLS LAST, CASE WHEN (lochierarchy#16 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#20 ASC NULLS FIRST], [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * ColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.date_dim (24) +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#20] +Output [2]: [d_date_sk#5, d_month_seq#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#20] -Condition : (((isnotnull(d_month_seq#20) AND (d_month_seq#20 >= 1200)) AND (d_month_seq#20 <= 1211)) AND isnotnull(d_date_sk#5)) +(27) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#21] +Condition : (((isnotnull(d_month_seq#21) AND (d_month_seq#21 >= 1200)) AND (d_month_seq#21 <= 1211)) AND isnotnull(d_date_sk#5)) -(26) CometProject -Input [2]: [d_date_sk#5, d_month_seq#20] +(28) CometProject +Input [2]: [d_date_sk#5, d_month_seq#21] Arguments: [d_date_sk#5], [d_date_sk#5] -(27) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(28) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index b4fcc80345..502cd9c52c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -1,41 +1,45 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (5) + WholeStageCodegen (6) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (4) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometSort [_w1,_w2,_w0] CometColumnarExchange [_w1,_w2] #1 - CometHashAggregate [i_category,i_class,spark_grouping_id,sum] - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [ws_net_paid,i_category,i_class] - Project [ws_net_paid,i_category,i_class] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] + Expand [ws_net_paid,i_category,i_class] + Project [ws_net_paid,i_category,i_class] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index 78435aff4b..14c482a3c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* ColumnarToRow (55) -+- CometHashAggregate (54) +* HashAggregate (55) ++- * ColumnarToRow (54) +- CometColumnarExchange (53) +- RowToColumnar (52) +- * HashAggregate (51) @@ -298,13 +298,15 @@ Input [1]: [count#26] Input [1]: [count#26] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(54) CometHashAggregate +(54) ColumnarToRow [codegen id : 13] +Input [1]: [count#26] + +(55) HashAggregate [codegen id : 13] Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] - -(55) ColumnarToRow [codegen id : 13] -Input [1]: [count(1)#27] +Aggregate Attributes [1]: [count(1)#27] +Results [1]: [count(1)#27 AS count(1)#28] ===== Subqueries ===== @@ -317,18 +319,18 @@ BroadcastExchange (60) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#28] +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct (57) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#28] -Condition : (((isnotnull(d_month_seq#28) AND (d_month_seq#28 >= 1200)) AND (d_month_seq#28 <= 1211)) AND isnotnull(d_date_sk#4)) +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) (58) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#28] +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] (59) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index 28771f9d42..c3e01e4f26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (13) - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),count(1),count] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (12) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index 0794d36e95..73c5b73f6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt @@ -6,8 +6,8 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (102) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (78) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (54) -: : : : : : :- * ColumnarToRow (30) -: : : : : : : +- CometHashAggregate (29) +: : : : : : :- * HashAggregate (30) +: : : : : : : +- * ColumnarToRow (29) : : : : : : : +- CometColumnarExchange (28) : : : : : : : +- RowToColumnar (27) : : : : : : : +- * HashAggregate (26) @@ -37,8 +37,8 @@ : : : : : : : +- CometFilter (20) : : : : : : : +- CometScan parquet spark_catalog.default.store (19) : : : : : : +- BroadcastExchange (53) -: : : : : : +- * ColumnarToRow (52) -: : : : : : +- CometHashAggregate (51) +: : : : : : +- * HashAggregate (52) +: : : : : : +- * ColumnarToRow (51) : : : : : : +- CometColumnarExchange (50) : : : : : : +- RowToColumnar (49) : : : : : : +- * HashAggregate (48) @@ -60,8 +60,8 @@ : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (38) : : : : : : +- ReusedExchange (45) : : : : : +- BroadcastExchange (77) -: : : : : +- * ColumnarToRow (76) -: : : : : +- CometHashAggregate (75) +: : : : : +- * HashAggregate (76) +: : : : : +- * ColumnarToRow (75) : : : : : +- CometColumnarExchange (74) : : : : : +- RowToColumnar (73) : : : : : +- * HashAggregate (72) @@ -83,8 +83,8 @@ : : : : : : +- CometScan parquet spark_catalog.default.time_dim (62) : : : : : +- ReusedExchange (69) : : : : +- BroadcastExchange (101) -: : : : +- * ColumnarToRow (100) -: : : : +- CometHashAggregate (99) +: : : : +- * HashAggregate (100) +: : : : +- * ColumnarToRow (99) : : : : +- CometColumnarExchange (98) : : : : +- RowToColumnar (97) : : : : +- * HashAggregate (96) @@ -106,8 +106,8 @@ : : : : : +- CometScan parquet spark_catalog.default.time_dim (86) : : : : +- ReusedExchange (93) : : : +- BroadcastExchange (125) -: : : +- * ColumnarToRow (124) -: : : +- CometHashAggregate (123) +: : : +- * HashAggregate (124) +: : : +- * ColumnarToRow (123) : : : +- CometColumnarExchange (122) : : : +- RowToColumnar (121) : : : +- * HashAggregate (120) @@ -129,8 +129,8 @@ : : : : +- CometScan parquet spark_catalog.default.time_dim (110) : : : +- ReusedExchange (117) : : +- BroadcastExchange (149) -: : +- * ColumnarToRow (148) -: : +- CometHashAggregate (147) +: : +- * HashAggregate (148) +: : +- * ColumnarToRow (147) : : +- CometColumnarExchange (146) : : +- RowToColumnar (145) : : +- * HashAggregate (144) @@ -152,8 +152,8 @@ : : : +- CometScan parquet spark_catalog.default.time_dim (134) : : +- ReusedExchange (141) : +- BroadcastExchange (173) -: +- * ColumnarToRow (172) -: +- CometHashAggregate (171) +: +- * HashAggregate (172) +: +- * ColumnarToRow (171) : +- CometColumnarExchange (170) : +- RowToColumnar (169) : +- * HashAggregate (168) @@ -175,8 +175,8 @@ : : +- CometScan parquet spark_catalog.default.time_dim (158) : +- ReusedExchange (165) +- BroadcastExchange (197) - +- * ColumnarToRow (196) - +- CometHashAggregate (195) + +- * HashAggregate (196) + +- * ColumnarToRow (195) +- CometColumnarExchange (194) +- RowToColumnar (193) +- * HashAggregate (192) @@ -327,114 +327,118 @@ Input [1]: [count#14] Input [1]: [count#14] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometHashAggregate +(29) ColumnarToRow [codegen id : 40] +Input [1]: [count#14] + +(30) HashAggregate [codegen id : 40] Input [1]: [count#14] Keys: [] Functions [1]: [count(1)] - -(30) ColumnarToRow [codegen id : 40] -Input [1]: [h8_30_to_9#15] +Aggregate Attributes [1]: [count(1)#15] +Results [1]: [count(1)#15 AS h8_30_to_9#16] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] +Output [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (32) CometFilter -Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] -Condition : ((isnotnull(ss_hdemo_sk#17) AND isnotnull(ss_sold_time_sk#16)) AND isnotnull(ss_store_sk#18)) +Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] +Condition : ((isnotnull(ss_hdemo_sk#18) AND isnotnull(ss_sold_time_sk#17)) AND isnotnull(ss_store_sk#19)) (33) CometProject -Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] -Arguments: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18], [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18] +Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] +Arguments: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19], [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] (34) ColumnarToRow [codegen id : 8] -Input [3]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18] +Input [3]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] (35) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#20] +Output [1]: [hd_demo_sk#21] (36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#17] -Right keys [1]: [hd_demo_sk#20] +Left keys [1]: [ss_hdemo_sk#18] +Right keys [1]: [hd_demo_sk#21] Join type: Inner Join condition: None (37) Project [codegen id : 8] -Output [2]: [ss_sold_time_sk#16, ss_store_sk#18] -Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, hd_demo_sk#20] +Output [2]: [ss_sold_time_sk#17, ss_store_sk#19] +Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, hd_demo_sk#21] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Output [3]: [t_time_sk#22, t_hour#23, t_minute#24] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (39) CometFilter -Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] -Condition : ((((isnotnull(t_hour#22) AND isnotnull(t_minute#23)) AND (t_hour#22 = 9)) AND (t_minute#23 < 30)) AND isnotnull(t_time_sk#21)) +Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] +Condition : ((((isnotnull(t_hour#23) AND isnotnull(t_minute#24)) AND (t_hour#23 = 9)) AND (t_minute#24 < 30)) AND isnotnull(t_time_sk#22)) (40) CometProject -Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] -Arguments: [t_time_sk#21], [t_time_sk#21] +Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] +Arguments: [t_time_sk#22], [t_time_sk#22] (41) ColumnarToRow [codegen id : 6] -Input [1]: [t_time_sk#21] +Input [1]: [t_time_sk#22] (42) BroadcastExchange -Input [1]: [t_time_sk#21] +Input [1]: [t_time_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] (43) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#16] -Right keys [1]: [t_time_sk#21] +Left keys [1]: [ss_sold_time_sk#17] +Right keys [1]: [t_time_sk#22] Join type: Inner Join condition: None (44) Project [codegen id : 8] -Output [1]: [ss_store_sk#18] -Input [3]: [ss_sold_time_sk#16, ss_store_sk#18, t_time_sk#21] +Output [1]: [ss_store_sk#19] +Input [3]: [ss_sold_time_sk#17, ss_store_sk#19, t_time_sk#22] (45) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#24] +Output [1]: [s_store_sk#25] (46) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#18] -Right keys [1]: [s_store_sk#24] +Left keys [1]: [ss_store_sk#19] +Right keys [1]: [s_store_sk#25] Join type: Inner Join condition: None (47) Project [codegen id : 8] Output: [] -Input [2]: [ss_store_sk#18, s_store_sk#24] +Input [2]: [ss_store_sk#19, s_store_sk#25] (48) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#25] -Results [1]: [count#26] +Aggregate Attributes [1]: [count#26] +Results [1]: [count#27] (49) RowToColumnar -Input [1]: [count#26] +Input [1]: [count#27] (50) CometColumnarExchange -Input [1]: [count#26] +Input [1]: [count#27] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(51) CometHashAggregate -Input [1]: [count#26] +(51) ColumnarToRow [codegen id : 9] +Input [1]: [count#27] + +(52) HashAggregate [codegen id : 9] +Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] - -(52) ColumnarToRow [codegen id : 9] -Input [1]: [h9_to_9_30#27] +Aggregate Attributes [1]: [count(1)#28] +Results [1]: [count(1)#28 AS h9_to_9_30#29] (53) BroadcastExchange -Input [1]: [h9_to_9_30#27] +Input [1]: [h9_to_9_30#29] Arguments: IdentityBroadcastMode, [plan_id=7] (54) BroadcastNestedLoopJoin [codegen id : 40] @@ -442,105 +446,107 @@ Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] +Output [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (56) CometFilter -Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] -Condition : ((isnotnull(ss_hdemo_sk#29) AND isnotnull(ss_sold_time_sk#28)) AND isnotnull(ss_store_sk#30)) +Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] +Condition : ((isnotnull(ss_hdemo_sk#31) AND isnotnull(ss_sold_time_sk#30)) AND isnotnull(ss_store_sk#32)) (57) CometProject -Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] -Arguments: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30], [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30] +Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] +Arguments: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32], [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] (58) ColumnarToRow [codegen id : 13] -Input [3]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30] +Input [3]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] (59) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#32] +Output [1]: [hd_demo_sk#34] (60) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#29] -Right keys [1]: [hd_demo_sk#32] +Left keys [1]: [ss_hdemo_sk#31] +Right keys [1]: [hd_demo_sk#34] Join type: Inner Join condition: None (61) Project [codegen id : 13] -Output [2]: [ss_sold_time_sk#28, ss_store_sk#30] -Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, hd_demo_sk#32] +Output [2]: [ss_sold_time_sk#30, ss_store_sk#32] +Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, hd_demo_sk#34] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Output [3]: [t_time_sk#35, t_hour#36, t_minute#37] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (63) CometFilter -Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] -Condition : ((((isnotnull(t_hour#34) AND isnotnull(t_minute#35)) AND (t_hour#34 = 9)) AND (t_minute#35 >= 30)) AND isnotnull(t_time_sk#33)) +Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] +Condition : ((((isnotnull(t_hour#36) AND isnotnull(t_minute#37)) AND (t_hour#36 = 9)) AND (t_minute#37 >= 30)) AND isnotnull(t_time_sk#35)) (64) CometProject -Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] -Arguments: [t_time_sk#33], [t_time_sk#33] +Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] +Arguments: [t_time_sk#35], [t_time_sk#35] (65) ColumnarToRow [codegen id : 11] -Input [1]: [t_time_sk#33] +Input [1]: [t_time_sk#35] (66) BroadcastExchange -Input [1]: [t_time_sk#33] +Input [1]: [t_time_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] (67) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#28] -Right keys [1]: [t_time_sk#33] +Left keys [1]: [ss_sold_time_sk#30] +Right keys [1]: [t_time_sk#35] Join type: Inner Join condition: None (68) Project [codegen id : 13] -Output [1]: [ss_store_sk#30] -Input [3]: [ss_sold_time_sk#28, ss_store_sk#30, t_time_sk#33] +Output [1]: [ss_store_sk#32] +Input [3]: [ss_sold_time_sk#30, ss_store_sk#32, t_time_sk#35] (69) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#36] +Output [1]: [s_store_sk#38] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#30] -Right keys [1]: [s_store_sk#36] +Left keys [1]: [ss_store_sk#32] +Right keys [1]: [s_store_sk#38] Join type: Inner Join condition: None (71) Project [codegen id : 13] Output: [] -Input [2]: [ss_store_sk#30, s_store_sk#36] +Input [2]: [ss_store_sk#32, s_store_sk#38] (72) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#37] -Results [1]: [count#38] +Aggregate Attributes [1]: [count#39] +Results [1]: [count#40] (73) RowToColumnar -Input [1]: [count#38] +Input [1]: [count#40] (74) CometColumnarExchange -Input [1]: [count#38] +Input [1]: [count#40] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(75) CometHashAggregate -Input [1]: [count#38] +(75) ColumnarToRow [codegen id : 14] +Input [1]: [count#40] + +(76) HashAggregate [codegen id : 14] +Input [1]: [count#40] Keys: [] Functions [1]: [count(1)] - -(76) ColumnarToRow [codegen id : 14] -Input [1]: [h9_30_to_10#39] +Aggregate Attributes [1]: [count(1)#41] +Results [1]: [count(1)#41 AS h9_30_to_10#42] (77) BroadcastExchange -Input [1]: [h9_30_to_10#39] +Input [1]: [h9_30_to_10#42] Arguments: IdentityBroadcastMode, [plan_id=10] (78) BroadcastNestedLoopJoin [codegen id : 40] @@ -548,105 +554,107 @@ Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] +Output [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (80) CometFilter -Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] -Condition : ((isnotnull(ss_hdemo_sk#41) AND isnotnull(ss_sold_time_sk#40)) AND isnotnull(ss_store_sk#42)) +Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] +Condition : ((isnotnull(ss_hdemo_sk#44) AND isnotnull(ss_sold_time_sk#43)) AND isnotnull(ss_store_sk#45)) (81) CometProject -Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] -Arguments: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42], [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42] +Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] +Arguments: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45], [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] (82) ColumnarToRow [codegen id : 18] -Input [3]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42] +Input [3]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] (83) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#44] +Output [1]: [hd_demo_sk#47] (84) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#41] -Right keys [1]: [hd_demo_sk#44] +Left keys [1]: [ss_hdemo_sk#44] +Right keys [1]: [hd_demo_sk#47] Join type: Inner Join condition: None (85) Project [codegen id : 18] -Output [2]: [ss_sold_time_sk#40, ss_store_sk#42] -Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, hd_demo_sk#44] +Output [2]: [ss_sold_time_sk#43, ss_store_sk#45] +Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, hd_demo_sk#47] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Output [3]: [t_time_sk#48, t_hour#49, t_minute#50] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (87) CometFilter -Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] -Condition : ((((isnotnull(t_hour#46) AND isnotnull(t_minute#47)) AND (t_hour#46 = 10)) AND (t_minute#47 < 30)) AND isnotnull(t_time_sk#45)) +Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] +Condition : ((((isnotnull(t_hour#49) AND isnotnull(t_minute#50)) AND (t_hour#49 = 10)) AND (t_minute#50 < 30)) AND isnotnull(t_time_sk#48)) (88) CometProject -Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] -Arguments: [t_time_sk#45], [t_time_sk#45] +Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] +Arguments: [t_time_sk#48], [t_time_sk#48] (89) ColumnarToRow [codegen id : 16] -Input [1]: [t_time_sk#45] +Input [1]: [t_time_sk#48] (90) BroadcastExchange -Input [1]: [t_time_sk#45] +Input [1]: [t_time_sk#48] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] (91) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#40] -Right keys [1]: [t_time_sk#45] +Left keys [1]: [ss_sold_time_sk#43] +Right keys [1]: [t_time_sk#48] Join type: Inner Join condition: None (92) Project [codegen id : 18] -Output [1]: [ss_store_sk#42] -Input [3]: [ss_sold_time_sk#40, ss_store_sk#42, t_time_sk#45] +Output [1]: [ss_store_sk#45] +Input [3]: [ss_sold_time_sk#43, ss_store_sk#45, t_time_sk#48] (93) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#48] +Output [1]: [s_store_sk#51] (94) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#42] -Right keys [1]: [s_store_sk#48] +Left keys [1]: [ss_store_sk#45] +Right keys [1]: [s_store_sk#51] Join type: Inner Join condition: None (95) Project [codegen id : 18] Output: [] -Input [2]: [ss_store_sk#42, s_store_sk#48] +Input [2]: [ss_store_sk#45, s_store_sk#51] (96) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#49] -Results [1]: [count#50] +Aggregate Attributes [1]: [count#52] +Results [1]: [count#53] (97) RowToColumnar -Input [1]: [count#50] +Input [1]: [count#53] (98) CometColumnarExchange -Input [1]: [count#50] +Input [1]: [count#53] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(99) CometHashAggregate -Input [1]: [count#50] +(99) ColumnarToRow [codegen id : 19] +Input [1]: [count#53] + +(100) HashAggregate [codegen id : 19] +Input [1]: [count#53] Keys: [] Functions [1]: [count(1)] - -(100) ColumnarToRow [codegen id : 19] -Input [1]: [h10_to_10_30#51] +Aggregate Attributes [1]: [count(1)#54] +Results [1]: [count(1)#54 AS h10_to_10_30#55] (101) BroadcastExchange -Input [1]: [h10_to_10_30#51] +Input [1]: [h10_to_10_30#55] Arguments: IdentityBroadcastMode, [plan_id=13] (102) BroadcastNestedLoopJoin [codegen id : 40] @@ -654,105 +662,107 @@ Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] +Output [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (104) CometFilter -Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] -Condition : ((isnotnull(ss_hdemo_sk#53) AND isnotnull(ss_sold_time_sk#52)) AND isnotnull(ss_store_sk#54)) +Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] +Condition : ((isnotnull(ss_hdemo_sk#57) AND isnotnull(ss_sold_time_sk#56)) AND isnotnull(ss_store_sk#58)) (105) CometProject -Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] -Arguments: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54], [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54] +Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] +Arguments: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58], [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] (106) ColumnarToRow [codegen id : 23] -Input [3]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54] +Input [3]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] (107) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#56] +Output [1]: [hd_demo_sk#60] (108) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#53] -Right keys [1]: [hd_demo_sk#56] +Left keys [1]: [ss_hdemo_sk#57] +Right keys [1]: [hd_demo_sk#60] Join type: Inner Join condition: None (109) Project [codegen id : 23] -Output [2]: [ss_sold_time_sk#52, ss_store_sk#54] -Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, hd_demo_sk#56] +Output [2]: [ss_sold_time_sk#56, ss_store_sk#58] +Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, hd_demo_sk#60] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Output [3]: [t_time_sk#61, t_hour#62, t_minute#63] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (111) CometFilter -Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] -Condition : ((((isnotnull(t_hour#58) AND isnotnull(t_minute#59)) AND (t_hour#58 = 10)) AND (t_minute#59 >= 30)) AND isnotnull(t_time_sk#57)) +Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] +Condition : ((((isnotnull(t_hour#62) AND isnotnull(t_minute#63)) AND (t_hour#62 = 10)) AND (t_minute#63 >= 30)) AND isnotnull(t_time_sk#61)) (112) CometProject -Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] -Arguments: [t_time_sk#57], [t_time_sk#57] +Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] +Arguments: [t_time_sk#61], [t_time_sk#61] (113) ColumnarToRow [codegen id : 21] -Input [1]: [t_time_sk#57] +Input [1]: [t_time_sk#61] (114) BroadcastExchange -Input [1]: [t_time_sk#57] +Input [1]: [t_time_sk#61] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] (115) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#52] -Right keys [1]: [t_time_sk#57] +Left keys [1]: [ss_sold_time_sk#56] +Right keys [1]: [t_time_sk#61] Join type: Inner Join condition: None (116) Project [codegen id : 23] -Output [1]: [ss_store_sk#54] -Input [3]: [ss_sold_time_sk#52, ss_store_sk#54, t_time_sk#57] +Output [1]: [ss_store_sk#58] +Input [3]: [ss_sold_time_sk#56, ss_store_sk#58, t_time_sk#61] (117) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#60] +Output [1]: [s_store_sk#64] (118) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#60] +Left keys [1]: [ss_store_sk#58] +Right keys [1]: [s_store_sk#64] Join type: Inner Join condition: None (119) Project [codegen id : 23] Output: [] -Input [2]: [ss_store_sk#54, s_store_sk#60] +Input [2]: [ss_store_sk#58, s_store_sk#64] (120) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#61] -Results [1]: [count#62] +Aggregate Attributes [1]: [count#65] +Results [1]: [count#66] (121) RowToColumnar -Input [1]: [count#62] +Input [1]: [count#66] (122) CometColumnarExchange -Input [1]: [count#62] +Input [1]: [count#66] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(123) CometHashAggregate -Input [1]: [count#62] +(123) ColumnarToRow [codegen id : 24] +Input [1]: [count#66] + +(124) HashAggregate [codegen id : 24] +Input [1]: [count#66] Keys: [] Functions [1]: [count(1)] - -(124) ColumnarToRow [codegen id : 24] -Input [1]: [h10_30_to_11#63] +Aggregate Attributes [1]: [count(1)#67] +Results [1]: [count(1)#67 AS h10_30_to_11#68] (125) BroadcastExchange -Input [1]: [h10_30_to_11#63] +Input [1]: [h10_30_to_11#68] Arguments: IdentityBroadcastMode, [plan_id=16] (126) BroadcastNestedLoopJoin [codegen id : 40] @@ -760,105 +770,107 @@ Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] +Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (128) CometFilter -Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] -Condition : ((isnotnull(ss_hdemo_sk#65) AND isnotnull(ss_sold_time_sk#64)) AND isnotnull(ss_store_sk#66)) +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) (129) CometProject -Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] -Arguments: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66], [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Arguments: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71], [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] (130) ColumnarToRow [codegen id : 28] -Input [3]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66] +Input [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] (131) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#68] +Output [1]: [hd_demo_sk#73] (132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#65] -Right keys [1]: [hd_demo_sk#68] +Left keys [1]: [ss_hdemo_sk#70] +Right keys [1]: [hd_demo_sk#73] Join type: Inner Join condition: None (133) Project [codegen id : 28] -Output [2]: [ss_sold_time_sk#64, ss_store_sk#66] -Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, hd_demo_sk#68] +Output [2]: [ss_sold_time_sk#69, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, hd_demo_sk#73] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Output [3]: [t_time_sk#74, t_hour#75, t_minute#76] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (135) CometFilter -Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] -Condition : ((((isnotnull(t_hour#70) AND isnotnull(t_minute#71)) AND (t_hour#70 = 11)) AND (t_minute#71 < 30)) AND isnotnull(t_time_sk#69)) +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Condition : ((((isnotnull(t_hour#75) AND isnotnull(t_minute#76)) AND (t_hour#75 = 11)) AND (t_minute#76 < 30)) AND isnotnull(t_time_sk#74)) (136) CometProject -Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] -Arguments: [t_time_sk#69], [t_time_sk#69] +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Arguments: [t_time_sk#74], [t_time_sk#74] (137) ColumnarToRow [codegen id : 26] -Input [1]: [t_time_sk#69] +Input [1]: [t_time_sk#74] (138) BroadcastExchange -Input [1]: [t_time_sk#69] +Input [1]: [t_time_sk#74] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] (139) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#64] -Right keys [1]: [t_time_sk#69] +Left keys [1]: [ss_sold_time_sk#69] +Right keys [1]: [t_time_sk#74] Join type: Inner Join condition: None (140) Project [codegen id : 28] -Output [1]: [ss_store_sk#66] -Input [3]: [ss_sold_time_sk#64, ss_store_sk#66, t_time_sk#69] +Output [1]: [ss_store_sk#71] +Input [3]: [ss_sold_time_sk#69, ss_store_sk#71, t_time_sk#74] (141) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#72] +Output [1]: [s_store_sk#77] (142) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#66] -Right keys [1]: [s_store_sk#72] +Left keys [1]: [ss_store_sk#71] +Right keys [1]: [s_store_sk#77] Join type: Inner Join condition: None (143) Project [codegen id : 28] Output: [] -Input [2]: [ss_store_sk#66, s_store_sk#72] +Input [2]: [ss_store_sk#71, s_store_sk#77] (144) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#73] -Results [1]: [count#74] +Aggregate Attributes [1]: [count#78] +Results [1]: [count#79] (145) RowToColumnar -Input [1]: [count#74] +Input [1]: [count#79] (146) CometColumnarExchange -Input [1]: [count#74] +Input [1]: [count#79] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(147) CometHashAggregate -Input [1]: [count#74] +(147) ColumnarToRow [codegen id : 29] +Input [1]: [count#79] + +(148) HashAggregate [codegen id : 29] +Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] - -(148) ColumnarToRow [codegen id : 29] -Input [1]: [h11_to_11_30#75] +Aggregate Attributes [1]: [count(1)#80] +Results [1]: [count(1)#80 AS h11_to_11_30#81] (149) BroadcastExchange -Input [1]: [h11_to_11_30#75] +Input [1]: [h11_to_11_30#81] Arguments: IdentityBroadcastMode, [plan_id=19] (150) BroadcastNestedLoopJoin [codegen id : 40] @@ -866,105 +878,107 @@ Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] +Output [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (152) CometFilter -Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] -Condition : ((isnotnull(ss_hdemo_sk#77) AND isnotnull(ss_sold_time_sk#76)) AND isnotnull(ss_store_sk#78)) +Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] +Condition : ((isnotnull(ss_hdemo_sk#83) AND isnotnull(ss_sold_time_sk#82)) AND isnotnull(ss_store_sk#84)) (153) CometProject -Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] -Arguments: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78], [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] +Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] +Arguments: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84], [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] (154) ColumnarToRow [codegen id : 33] -Input [3]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] +Input [3]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] (155) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#80] +Output [1]: [hd_demo_sk#86] (156) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#77] -Right keys [1]: [hd_demo_sk#80] +Left keys [1]: [ss_hdemo_sk#83] +Right keys [1]: [hd_demo_sk#86] Join type: Inner Join condition: None (157) Project [codegen id : 33] -Output [2]: [ss_sold_time_sk#76, ss_store_sk#78] -Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, hd_demo_sk#80] +Output [2]: [ss_sold_time_sk#82, ss_store_sk#84] +Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, hd_demo_sk#86] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Output [3]: [t_time_sk#87, t_hour#88, t_minute#89] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (159) CometFilter -Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] -Condition : ((((isnotnull(t_hour#82) AND isnotnull(t_minute#83)) AND (t_hour#82 = 11)) AND (t_minute#83 >= 30)) AND isnotnull(t_time_sk#81)) +Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] +Condition : ((((isnotnull(t_hour#88) AND isnotnull(t_minute#89)) AND (t_hour#88 = 11)) AND (t_minute#89 >= 30)) AND isnotnull(t_time_sk#87)) (160) CometProject -Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] -Arguments: [t_time_sk#81], [t_time_sk#81] +Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] +Arguments: [t_time_sk#87], [t_time_sk#87] (161) ColumnarToRow [codegen id : 31] -Input [1]: [t_time_sk#81] +Input [1]: [t_time_sk#87] (162) BroadcastExchange -Input [1]: [t_time_sk#81] +Input [1]: [t_time_sk#87] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] (163) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#76] -Right keys [1]: [t_time_sk#81] +Left keys [1]: [ss_sold_time_sk#82] +Right keys [1]: [t_time_sk#87] Join type: Inner Join condition: None (164) Project [codegen id : 33] -Output [1]: [ss_store_sk#78] -Input [3]: [ss_sold_time_sk#76, ss_store_sk#78, t_time_sk#81] +Output [1]: [ss_store_sk#84] +Input [3]: [ss_sold_time_sk#82, ss_store_sk#84, t_time_sk#87] (165) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#84] +Output [1]: [s_store_sk#90] (166) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#78] -Right keys [1]: [s_store_sk#84] +Left keys [1]: [ss_store_sk#84] +Right keys [1]: [s_store_sk#90] Join type: Inner Join condition: None (167) Project [codegen id : 33] Output: [] -Input [2]: [ss_store_sk#78, s_store_sk#84] +Input [2]: [ss_store_sk#84, s_store_sk#90] (168) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#85] -Results [1]: [count#86] +Aggregate Attributes [1]: [count#91] +Results [1]: [count#92] (169) RowToColumnar -Input [1]: [count#86] +Input [1]: [count#92] (170) CometColumnarExchange -Input [1]: [count#86] +Input [1]: [count#92] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=21] -(171) CometHashAggregate -Input [1]: [count#86] +(171) ColumnarToRow [codegen id : 34] +Input [1]: [count#92] + +(172) HashAggregate [codegen id : 34] +Input [1]: [count#92] Keys: [] Functions [1]: [count(1)] - -(172) ColumnarToRow [codegen id : 34] -Input [1]: [h11_30_to_12#87] +Aggregate Attributes [1]: [count(1)#93] +Results [1]: [count(1)#93 AS h11_30_to_12#94] (173) BroadcastExchange -Input [1]: [h11_30_to_12#87] +Input [1]: [h11_30_to_12#94] Arguments: IdentityBroadcastMode, [plan_id=22] (174) BroadcastNestedLoopJoin [codegen id : 40] @@ -972,105 +986,107 @@ Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] +Output [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (176) CometFilter -Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] -Condition : ((isnotnull(ss_hdemo_sk#89) AND isnotnull(ss_sold_time_sk#88)) AND isnotnull(ss_store_sk#90)) +Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] +Condition : ((isnotnull(ss_hdemo_sk#96) AND isnotnull(ss_sold_time_sk#95)) AND isnotnull(ss_store_sk#97)) (177) CometProject -Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] -Arguments: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90], [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90] +Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] +Arguments: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97], [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] (178) ColumnarToRow [codegen id : 38] -Input [3]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90] +Input [3]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] (179) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#92] +Output [1]: [hd_demo_sk#99] (180) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#89] -Right keys [1]: [hd_demo_sk#92] +Left keys [1]: [ss_hdemo_sk#96] +Right keys [1]: [hd_demo_sk#99] Join type: Inner Join condition: None (181) Project [codegen id : 38] -Output [2]: [ss_sold_time_sk#88, ss_store_sk#90] -Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, hd_demo_sk#92] +Output [2]: [ss_sold_time_sk#95, ss_store_sk#97] +Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, hd_demo_sk#99] (unknown) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Output [3]: [t_time_sk#100, t_hour#101, t_minute#102] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (183) CometFilter -Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] -Condition : ((((isnotnull(t_hour#94) AND isnotnull(t_minute#95)) AND (t_hour#94 = 12)) AND (t_minute#95 < 30)) AND isnotnull(t_time_sk#93)) +Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] +Condition : ((((isnotnull(t_hour#101) AND isnotnull(t_minute#102)) AND (t_hour#101 = 12)) AND (t_minute#102 < 30)) AND isnotnull(t_time_sk#100)) (184) CometProject -Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] -Arguments: [t_time_sk#93], [t_time_sk#93] +Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] +Arguments: [t_time_sk#100], [t_time_sk#100] (185) ColumnarToRow [codegen id : 36] -Input [1]: [t_time_sk#93] +Input [1]: [t_time_sk#100] (186) BroadcastExchange -Input [1]: [t_time_sk#93] +Input [1]: [t_time_sk#100] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23] (187) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#88] -Right keys [1]: [t_time_sk#93] +Left keys [1]: [ss_sold_time_sk#95] +Right keys [1]: [t_time_sk#100] Join type: Inner Join condition: None (188) Project [codegen id : 38] -Output [1]: [ss_store_sk#90] -Input [3]: [ss_sold_time_sk#88, ss_store_sk#90, t_time_sk#93] +Output [1]: [ss_store_sk#97] +Input [3]: [ss_sold_time_sk#95, ss_store_sk#97, t_time_sk#100] (189) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#96] +Output [1]: [s_store_sk#103] (190) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#90] -Right keys [1]: [s_store_sk#96] +Left keys [1]: [ss_store_sk#97] +Right keys [1]: [s_store_sk#103] Join type: Inner Join condition: None (191) Project [codegen id : 38] Output: [] -Input [2]: [ss_store_sk#90, s_store_sk#96] +Input [2]: [ss_store_sk#97, s_store_sk#103] (192) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#97] -Results [1]: [count#98] +Aggregate Attributes [1]: [count#104] +Results [1]: [count#105] (193) RowToColumnar -Input [1]: [count#98] +Input [1]: [count#105] (194) CometColumnarExchange -Input [1]: [count#98] +Input [1]: [count#105] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=24] -(195) CometHashAggregate -Input [1]: [count#98] +(195) ColumnarToRow [codegen id : 39] +Input [1]: [count#105] + +(196) HashAggregate [codegen id : 39] +Input [1]: [count#105] Keys: [] Functions [1]: [count(1)] - -(196) ColumnarToRow [codegen id : 39] -Input [1]: [h12_to_12_30#99] +Aggregate Attributes [1]: [count(1)#106] +Results [1]: [count(1)#106 AS h12_to_12_30#107] (197) BroadcastExchange -Input [1]: [h12_to_12_30#99] +Input [1]: [h12_to_12_30#107] Arguments: IdentityBroadcastMode, [plan_id=25] (198) BroadcastNestedLoopJoin [codegen id : 40] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index b80f512c53..85d520e7d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -6,9 +6,9 @@ WholeStageCodegen (40) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),h8_30_to_9,count] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (4) @@ -51,9 +51,9 @@ WholeStageCodegen (40) InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),h9_to_9_30,count] + ColumnarToRow + InputAdapter CometColumnarExchange #6 RowToColumnar WholeStageCodegen (8) @@ -84,9 +84,9 @@ WholeStageCodegen (40) InputAdapter BroadcastExchange #8 WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),h9_30_to_10,count] + ColumnarToRow + InputAdapter CometColumnarExchange #9 RowToColumnar WholeStageCodegen (13) @@ -117,9 +117,9 @@ WholeStageCodegen (40) InputAdapter BroadcastExchange #11 WholeStageCodegen (19) - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),h10_to_10_30,count] + ColumnarToRow + InputAdapter CometColumnarExchange #12 RowToColumnar WholeStageCodegen (18) @@ -150,9 +150,9 @@ WholeStageCodegen (40) InputAdapter BroadcastExchange #14 WholeStageCodegen (24) - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),h10_30_to_11,count] + ColumnarToRow + InputAdapter CometColumnarExchange #15 RowToColumnar WholeStageCodegen (23) @@ -183,9 +183,9 @@ WholeStageCodegen (40) InputAdapter BroadcastExchange #17 WholeStageCodegen (29) - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),h11_to_11_30,count] + ColumnarToRow + InputAdapter CometColumnarExchange #18 RowToColumnar WholeStageCodegen (28) @@ -216,9 +216,9 @@ WholeStageCodegen (40) InputAdapter BroadcastExchange #20 WholeStageCodegen (34) - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),h11_30_to_12,count] + ColumnarToRow + InputAdapter CometColumnarExchange #21 RowToColumnar WholeStageCodegen (33) @@ -249,9 +249,9 @@ WholeStageCodegen (40) InputAdapter BroadcastExchange #23 WholeStageCodegen (39) - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),h12_to_12_30,count] + ColumnarToRow + InputAdapter CometColumnarExchange #24 RowToColumnar WholeStageCodegen (38) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index 66c4896f4c..d7834e74c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -1,33 +1,35 @@ == Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * ColumnarToRow (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- CometHashAggregate (22) - +- CometColumnarExchange (21) - +- RowToColumnar (20) - +- * HashAggregate (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.store_sales (4) - : +- ReusedExchange (10) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store (13) +TakeOrderedAndProject (31) ++- * Project (30) + +- * Filter (29) + +- Window (28) + +- * ColumnarToRow (27) + +- CometSort (26) + +- CometColumnarExchange (25) + +- RowToColumnar (24) + +- * HashAggregate (23) + +- * ColumnarToRow (22) + +- CometColumnarExchange (21) + +- RowToColumnar (20) + +- * HashAggregate (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.store_sales (4) + : +- ReusedExchange (10) + +- BroadcastExchange (16) + +- * ColumnarToRow (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.store (13) (unknown) Scan parquet spark_catalog.default.item @@ -73,7 +75,7 @@ Join condition: None Output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] Input [8]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] -(10) ReusedExchange [Reuses operator id: 34] +(10) ReusedExchange [Reuses operator id: 36] Output [2]: [d_date_sk#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -128,67 +130,75 @@ Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name# Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] Arguments: hashpartitioning(i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometHashAggregate +(22) ColumnarToRow [codegen id : 5] +Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] + +(23) HashAggregate [codegen id : 5] Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#7))#17] +Results [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS _w0#19] + +(24) RowToColumnar +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -(23) CometColumnarExchange -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18] +(25) CometColumnarExchange +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometSort -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18] -Arguments: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18], [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST] +(26) CometSort +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] +Arguments: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19], [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 5] -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18] +(27) ColumnarToRow [codegen id : 6] +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -(26) Window -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#19], [i_category#4, i_brand#2, s_store_name#13, s_company_name#14] +(28) Window +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#4, i_brand#2, s_store_name#13, s_company_name#14] -(27) Filter [codegen id : 6] -Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18, avg_monthly_sales#19] -Condition : CASE WHEN NOT (avg_monthly_sales#19 = 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#19)) / avg_monthly_sales#19) > 0.1000000000000000) END +(29) Filter [codegen id : 7] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] +Condition : CASE WHEN NOT (avg_monthly_sales#20 = 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END -(28) Project [codegen id : 6] -Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, avg_monthly_sales#19] -Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, _w0#18, avg_monthly_sales#19] +(30) Project [codegen id : 7] +Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] -(29) TakeOrderedAndProject -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, avg_monthly_sales#19] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#19) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#17, avg_monthly_sales#19] +(31) TakeOrderedAndProject +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (34) -+- * ColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.date_dim (30) +BroadcastExchange (36) ++- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#20, d_moy#11] +Output [3]: [d_date_sk#10, d_year#21, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(31) CometFilter -Input [3]: [d_date_sk#10, d_year#20, d_moy#11] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#10)) +(33) CometFilter +Input [3]: [d_date_sk#10, d_year#21, d_moy#11] +Condition : ((isnotnull(d_year#21) AND (d_year#21 = 1999)) AND isnotnull(d_date_sk#10)) -(32) CometProject -Input [3]: [d_date_sk#10, d_year#20, d_moy#11] +(34) CometProject +Input [3]: [d_date_sk#10, d_year#21, d_moy#11] Arguments: [d_date_sk#10, d_moy#11], [d_date_sk#10, d_moy#11] -(33) ColumnarToRow [codegen id : 1] +(35) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_moy#11] -(34) BroadcastExchange +(36) BroadcastExchange Input [2]: [d_date_sk#10, d_moy#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index 8007b3089e..38e228738e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -1,50 +1,54 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (6) + WholeStageCodegen (7) Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name] CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_category,i_class,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_category,i_class,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #4 InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt index f75f1c1908..8931917005 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == * Project (55) +- * BroadcastNestedLoopJoin Inner BuildRight (54) - :- * ColumnarToRow (30) - : +- CometHashAggregate (29) + :- * HashAggregate (30) + : +- * ColumnarToRow (29) : +- CometColumnarExchange (28) : +- RowToColumnar (27) : +- * HashAggregate (26) @@ -32,8 +32,8 @@ : +- CometFilter (20) : +- CometScan parquet spark_catalog.default.web_page (19) +- BroadcastExchange (53) - +- * ColumnarToRow (52) - +- CometHashAggregate (51) + +- * HashAggregate (52) + +- * ColumnarToRow (51) +- CometColumnarExchange (50) +- RowToColumnar (49) +- * HashAggregate (48) @@ -184,114 +184,118 @@ Input [1]: [count#12] Input [1]: [count#12] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometHashAggregate +(29) ColumnarToRow [codegen id : 10] +Input [1]: [count#12] + +(30) HashAggregate [codegen id : 10] Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] - -(30) ColumnarToRow [codegen id : 10] -Input [1]: [amc#13] +Aggregate Attributes [1]: [count(1)#13] +Results [1]: [count(1)#13 AS amc#14] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] +Output [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct (32) CometFilter -Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] -Condition : ((isnotnull(ws_ship_hdemo_sk#15) AND isnotnull(ws_sold_time_sk#14)) AND isnotnull(ws_web_page_sk#16)) +Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] +Condition : ((isnotnull(ws_ship_hdemo_sk#16) AND isnotnull(ws_sold_time_sk#15)) AND isnotnull(ws_web_page_sk#17)) (33) CometProject -Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] -Arguments: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16], [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16] +Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] +Arguments: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17], [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] (34) ColumnarToRow [codegen id : 8] -Input [3]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16] +Input [3]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] (35) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#18] +Output [1]: [hd_demo_sk#19] (36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#15] -Right keys [1]: [hd_demo_sk#18] +Left keys [1]: [ws_ship_hdemo_sk#16] +Right keys [1]: [hd_demo_sk#19] Join type: Inner Join condition: None (37) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#14, ws_web_page_sk#16] -Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, hd_demo_sk#18] +Output [2]: [ws_sold_time_sk#15, ws_web_page_sk#17] +Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, hd_demo_sk#19] (unknown) Scan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#19, t_hour#20] +Output [2]: [t_time_sk#20, t_hour#21] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct (39) CometFilter -Input [2]: [t_time_sk#19, t_hour#20] -Condition : (((isnotnull(t_hour#20) AND (t_hour#20 >= 19)) AND (t_hour#20 <= 20)) AND isnotnull(t_time_sk#19)) +Input [2]: [t_time_sk#20, t_hour#21] +Condition : (((isnotnull(t_hour#21) AND (t_hour#21 >= 19)) AND (t_hour#21 <= 20)) AND isnotnull(t_time_sk#20)) (40) CometProject -Input [2]: [t_time_sk#19, t_hour#20] -Arguments: [t_time_sk#19], [t_time_sk#19] +Input [2]: [t_time_sk#20, t_hour#21] +Arguments: [t_time_sk#20], [t_time_sk#20] (41) ColumnarToRow [codegen id : 6] -Input [1]: [t_time_sk#19] +Input [1]: [t_time_sk#20] (42) BroadcastExchange -Input [1]: [t_time_sk#19] +Input [1]: [t_time_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] (43) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#14] -Right keys [1]: [t_time_sk#19] +Left keys [1]: [ws_sold_time_sk#15] +Right keys [1]: [t_time_sk#20] Join type: Inner Join condition: None (44) Project [codegen id : 8] -Output [1]: [ws_web_page_sk#16] -Input [3]: [ws_sold_time_sk#14, ws_web_page_sk#16, t_time_sk#19] +Output [1]: [ws_web_page_sk#17] +Input [3]: [ws_sold_time_sk#15, ws_web_page_sk#17, t_time_sk#20] (45) ReusedExchange [Reuses operator id: 23] -Output [1]: [wp_web_page_sk#21] +Output [1]: [wp_web_page_sk#22] (46) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#16] -Right keys [1]: [wp_web_page_sk#21] +Left keys [1]: [ws_web_page_sk#17] +Right keys [1]: [wp_web_page_sk#22] Join type: Inner Join condition: None (47) Project [codegen id : 8] Output: [] -Input [2]: [ws_web_page_sk#16, wp_web_page_sk#21] +Input [2]: [ws_web_page_sk#17, wp_web_page_sk#22] (48) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#22] -Results [1]: [count#23] +Aggregate Attributes [1]: [count#23] +Results [1]: [count#24] (49) RowToColumnar -Input [1]: [count#23] +Input [1]: [count#24] (50) CometColumnarExchange -Input [1]: [count#23] +Input [1]: [count#24] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(51) CometHashAggregate -Input [1]: [count#23] +(51) ColumnarToRow [codegen id : 9] +Input [1]: [count#24] + +(52) HashAggregate [codegen id : 9] +Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] - -(52) ColumnarToRow [codegen id : 9] -Input [1]: [pmc#24] +Aggregate Attributes [1]: [count(1)#25] +Results [1]: [count(1)#25 AS pmc#26] (53) BroadcastExchange -Input [1]: [pmc#24] +Input [1]: [pmc#26] Arguments: IdentityBroadcastMode, [plan_id=7] (54) BroadcastNestedLoopJoin [codegen id : 10] @@ -299,6 +303,6 @@ Join type: Inner Join condition: None (55) Project [codegen id : 10] -Output [1]: [(cast(amc#13 as decimal(15,4)) / cast(pmc#24 as decimal(15,4))) AS am_pm_ratio#25] -Input [2]: [amc#13, pmc#24] +Output [1]: [(cast(amc#14 as decimal(15,4)) / cast(pmc#26 as decimal(15,4))) AS am_pm_ratio#27] +Input [2]: [amc#14, pmc#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index 61f38f3615..d9cfe34cab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -1,9 +1,9 @@ WholeStageCodegen (10) Project [amc,pmc] BroadcastNestedLoopJoin - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),amc,count] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (4) @@ -46,9 +46,9 @@ WholeStageCodegen (10) InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),pmc,count] + ColumnarToRow + InputAdapter CometColumnarExchange #6 RowToColumnar WholeStageCodegen (8) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt index 2d0c9fd987..0a57a5b1f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt @@ -1,49 +1,51 @@ == Physical Plan == -* ColumnarToRow (45) -+- CometSort (44) - +- CometColumnarExchange (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- RowToColumnar (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * Project (12) - : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.call_center (1) - : : : : : +- BroadcastExchange (7) - : : : : : +- * ColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (4) - : : : : +- ReusedExchange (10) - : : : +- BroadcastExchange (16) - : : : +- * ColumnarToRow (15) - : : : +- CometFilter (14) - : : : +- CometScan parquet spark_catalog.default.customer (13) - : : +- BroadcastExchange (23) - : : +- * ColumnarToRow (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan parquet spark_catalog.default.customer_address (19) - : +- BroadcastExchange (29) - : +- * ColumnarToRow (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.customer_demographics (26) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.household_demographics (32) +* ColumnarToRow (47) ++- CometSort (46) + +- CometColumnarExchange (45) + +- RowToColumnar (44) + +- * HashAggregate (43) + +- * ColumnarToRow (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (18) + : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : :- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * Project (9) + : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.call_center (1) + : : : : : +- BroadcastExchange (7) + : : : : : +- * ColumnarToRow (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (4) + : : : : +- ReusedExchange (10) + : : : +- BroadcastExchange (16) + : : : +- * ColumnarToRow (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.customer (13) + : : +- BroadcastExchange (23) + : : +- * ColumnarToRow (22) + : : +- CometProject (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.customer_address (19) + : +- BroadcastExchange (29) + : +- * ColumnarToRow (28) + : +- CometFilter (27) + : +- CometScan parquet spark_catalog.default.customer_demographics (26) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.household_demographics (32) (unknown) Scan parquet spark_catalog.default.call_center @@ -89,7 +91,7 @@ Join condition: None Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8] Input [8]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] -(10) ReusedExchange [Reuses operator id: 50] +(10) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#10] (11) BroadcastHashJoin [codegen id : 7] @@ -236,51 +238,59 @@ Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] Arguments: hashpartitioning(cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(42) CometHashAggregate +(42) ColumnarToRow [codegen id : 8] +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] + +(43) HashAggregate [codegen id : 8] Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19] Functions [1]: [sum(UnscaledValue(cr_net_loss#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#7))#24] +Results [4]: [cc_call_center_id#2 AS Call_Center#25, cc_name#3 AS Call_Center_Name#26, cc_manager#4 AS Manager#27, MakeDecimal(sum(UnscaledValue(cr_net_loss#7))#24,17,2) AS Returns_Loss#28] + +(44) RowToColumnar +Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] -(43) CometColumnarExchange -Input [4]: [Call_Center#24, Call_Center_Name#25, Manager#26, Returns_Loss#27] -Arguments: rangepartitioning(Returns_Loss#27 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(45) CometColumnarExchange +Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] +Arguments: rangepartitioning(Returns_Loss#28 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(44) CometSort -Input [4]: [Call_Center#24, Call_Center_Name#25, Manager#26, Returns_Loss#27] -Arguments: [Call_Center#24, Call_Center_Name#25, Manager#26, Returns_Loss#27], [Returns_Loss#27 DESC NULLS LAST] +(46) CometSort +Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] +Arguments: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28], [Returns_Loss#28 DESC NULLS LAST] -(45) ColumnarToRow [codegen id : 8] -Input [4]: [Call_Center#24, Call_Center_Name#25, Manager#26, Returns_Loss#27] +(47) ColumnarToRow [codegen id : 9] +Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#28, d_moy#29] +Output [3]: [d_date_sk#10, d_year#29, d_moy#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [3]: [d_date_sk#10, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 1998)) AND (d_moy#29 = 11)) AND isnotnull(d_date_sk#10)) +(49) CometFilter +Input [3]: [d_date_sk#10, d_year#29, d_moy#30] +Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 1998)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#10)) -(48) CometProject -Input [3]: [d_date_sk#10, d_year#28, d_moy#29] +(50) CometProject +Input [3]: [d_date_sk#10, d_year#29, d_moy#30] Arguments: [d_date_sk#10], [d_date_sk#10] -(49) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(50) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index c2b89020cb..75273fafec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -1,73 +1,77 @@ -WholeStageCodegen (8) +WholeStageCodegen (9) ColumnarToRow InputAdapter CometSort [Returns_Loss] CometColumnarExchange [Returns_Loss] #1 - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] - CometColumnarExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - ColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + RowToColumnar + WholeStageCodegen (8) + HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] + BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] + ColumnarToRow + InputAdapter + CometFilter [cc_call_center_sk] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [cr_call_center_sk,cr_returning_customer_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] InputAdapter - ReusedExchange [d_date_sk] #4 + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #7 + WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #8 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index f4d3879699..04fa7a1bac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* ColumnarToRow (33) -+- CometHashAggregate (32) +* HashAggregate (33) ++- * ColumnarToRow (32) +- CometColumnarExchange (31) +- RowToColumnar (30) +- * HashAggregate (29) @@ -19,9 +19,9 @@ : : +- CometFilter (5) : : +- CometScan parquet spark_catalog.default.item (4) : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometFilter (21) - : +- CometHashAggregate (20) + : +- * Filter (22) + : +- * HashAggregate (21) + : +- * ColumnarToRow (20) : +- CometColumnarExchange (19) : +- RowToColumnar (18) : +- * HashAggregate (17) @@ -123,66 +123,70 @@ Input [3]: [ws_item_sk#7, sum#14, count#15] Input [3]: [ws_item_sk#7, sum#14, count#15] Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(20) CometHashAggregate +(20) ColumnarToRow [codegen id : 4] +Input [3]: [ws_item_sk#7, sum#14, count#15] + +(21) HashAggregate [codegen id : 4] Input [3]: [ws_item_sk#7, sum#14, count#15] Keys [1]: [ws_item_sk#7] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#16] +Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] -(21) CometFilter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) - -(22) ColumnarToRow [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] +(22) Filter [codegen id : 4] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) (23) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] Right keys [1]: [ws_item_sk#7] Join type: Inner -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#16) +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) (25) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] (26) ReusedExchange [Reuses operator id: 38] -Output [1]: [d_date_sk#17] +Output [1]: [d_date_sk#18] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None (28) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#17] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#18] (29) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#18] -Results [1]: [sum#19] +Aggregate Attributes [1]: [sum#19] +Results [1]: [sum#20] (30) RowToColumnar -Input [1]: [sum#19] +Input [1]: [sum#20] (31) CometColumnarExchange -Input [1]: [sum#19] +Input [1]: [sum#20] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(32) CometHashAggregate -Input [1]: [sum#19] +(32) ColumnarToRow [codegen id : 7] +Input [1]: [sum#20] + +(33) HashAggregate [codegen id : 7] +Input [1]: [sum#20] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] - -(33) ColumnarToRow [codegen id : 7] -Input [1]: [Excess Discount Amount #20] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#21] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#21,17,2) AS Excess Discount Amount #22] ===== Subqueries ===== @@ -195,25 +199,25 @@ BroadcastExchange (38) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_date#21] +Output [2]: [d_date_sk#18, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (35) CometFilter -Input [2]: [d_date_sk#17, d_date#21] -Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 2000-01-27)) AND (d_date#21 <= 2000-04-26)) AND isnotnull(d_date_sk#17)) +Input [2]: [d_date_sk#18, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) (36) CometProject -Input [2]: [d_date_sk#17, d_date#21] -Arguments: [d_date_sk#17], [d_date_sk#17] +Input [2]: [d_date_sk#18, d_date#23] +Arguments: [d_date_sk#18], [d_date_sk#18] (37) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] +Input [1]: [d_date_sk#18] (38) BroadcastExchange -Input [1]: [d_date_sk#17] +Input [1]: [d_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index 3ef3c57493..3b87e0d3ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [sum] + HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (6) @@ -35,10 +35,10 @@ WholeStageCodegen (7) InputAdapter BroadcastExchange #4 WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [(1.3 * avg(ws_ext_discount_amt))] - CometHashAggregate [ws_item_sk,sum,count] + Filter [(1.3 * avg(ws_ext_discount_amt))] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ws_item_sk] #5 RowToColumnar WholeStageCodegen (3) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index c7490b740f..5cf504d2d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- CometHashAggregate (23) +TakeOrderedAndProject (25) ++- * HashAggregate (24) + +- * ColumnarToRow (23) +- CometColumnarExchange (22) +- RowToColumnar (21) +- * HashAggregate (20) @@ -125,15 +125,17 @@ Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(23) CometHashAggregate +(23) ColumnarToRow [codegen id : 3] +Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] + +(24) HashAggregate [codegen id : 3] Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Keys [1]: [ss_customer_sk#2] Functions [1]: [sum(act_sales#14)] +Aggregate Attributes [1]: [sum(act_sales#14)#19] +Results [2]: [ss_customer_sk#2, sum(act_sales#14)#19 AS sumsales#20] -(24) CometTakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#19] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#19 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#19]), 100, [sumsales#19 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#19] - -(25) ColumnarToRow [codegen id : 3] -Input [2]: [ss_customer_sk#2, sumsales#19] +(25) TakeOrderedAndProject +Input [2]: [ss_customer_sk#2, sumsales#20] +Arguments: 100, [sumsales#20 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#20] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index 85a37565ff..fbd6290f3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [sumsales,ss_customer_sk] - CometHashAggregate [ss_customer_sk,sum,isEmpty] +TakeOrderedAndProject [sumsales,ss_customer_sk] + WholeStageCodegen (3) + HashAggregate [ss_customer_sk,sum,isEmpty] [sum(act_sales),sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter CometColumnarExchange [ss_customer_sk] #1 RowToColumnar WholeStageCodegen (2) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 569baa88c6..e48a84a6fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* ColumnarToRow (47) -+- CometHashAggregate (46) +* HashAggregate (47) ++- * ColumnarToRow (46) +- CometColumnarExchange (45) +- RowToColumnar (44) +- * HashAggregate (43) @@ -256,11 +256,13 @@ Input [3]: [sum#22, sum#23, count#25] Input [3]: [sum#22, sum#23, count#25] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(46) CometHashAggregate +(46) ColumnarToRow [codegen id : 9] +Input [3]: [sum#22, sum#23, count#25] + +(47) HashAggregate [codegen id : 9] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] - -(47) ColumnarToRow [codegen id : 9] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21, count(ws_order_number#5)#24] +Results [3]: [count(ws_order_number#5)#24 AS order count #26, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#20,17,2) AS total shipping cost #27, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#21,17,2) AS total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 1784692e7c..cb36bcd9a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] + HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (8) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index c9e31178a9..45fe92329b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* ColumnarToRow (63) -+- CometHashAggregate (62) +* HashAggregate (63) ++- * ColumnarToRow (62) +- CometColumnarExchange (61) +- RowToColumnar (60) +- * HashAggregate (59) @@ -338,11 +338,13 @@ Input [3]: [sum#23, sum#24, count#26] Input [3]: [sum#23, sum#24, count#26] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(62) CometHashAggregate +(62) ColumnarToRow [codegen id : 15] +Input [3]: [sum#23, sum#24, count#26] + +(63) HashAggregate [codegen id : 15] Input [3]: [sum#23, sum#24, count#26] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] - -(63) ColumnarToRow [codegen id : 15] -Input [3]: [order count #27, total shipping cost #28, total net profit #29] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22, count(ws_order_number#4)#25] +Results [3]: [count(ws_order_number#4)#25 AS order count #27, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#21,17,2) AS total shipping cost #28, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#22,17,2) AS total net profit #29] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 335ef0af24..704732744e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (15) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] + HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (14) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt index 49062fc63a..c5815879b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* ColumnarToRow (30) -+- CometHashAggregate (29) +* HashAggregate (30) ++- * ColumnarToRow (29) +- CometColumnarExchange (28) +- RowToColumnar (27) +- * HashAggregate (26) @@ -159,11 +159,13 @@ Input [1]: [count#13] Input [1]: [count#13] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometHashAggregate +(29) ColumnarToRow [codegen id : 5] +Input [1]: [count#13] + +(30) HashAggregate [codegen id : 5] Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] - -(30) ColumnarToRow [codegen id : 5] -Input [1]: [count(1)#14] +Aggregate Attributes [1]: [count(1)#14] +Results [1]: [count(1)#14 AS count(1)#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt index 647015f01c..50ec5cdde4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [count] + HashAggregate [count] [count(1),count(1),count] + ColumnarToRow + InputAdapter CometColumnarExchange #1 RowToColumnar WholeStageCodegen (4) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index 457e702c53..1d9b5d4bab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -1,30 +1,32 @@ == Physical Plan == -* ColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- RowToColumnar (22) - +- * Project (21) - +- Window (20) - +- * ColumnarToRow (19) - +- CometSort (18) - +- CometColumnarExchange (17) - +- CometHashAggregate (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +* ColumnarToRow (28) ++- CometProject (27) + +- CometSort (26) + +- CometColumnarExchange (25) + +- RowToColumnar (24) + +- * Project (23) + +- Window (22) + +- * ColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- RowToColumnar (18) + +- * HashAggregate (17) + +- * ColumnarToRow (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.store_sales @@ -70,7 +72,7 @@ Join condition: None Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 31] +(10) ReusedExchange [Reuses operator id: 33] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -97,77 +99,85 @@ Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] +Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] + +(18) RowToColumnar +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -(17) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +(19) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) CometSort -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] -Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6], [i_class#9 ASC NULLS FIRST] +(20) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 4] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] +(21) ColumnarToRow [codegen id : 5] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -(20) Window -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6] -Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] +(22) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(21) Project [codegen id : 5] -Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17, i_item_id#6] -Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, i_item_id#6, _we0#16] +(23) Project [codegen id : 6] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] -(22) RowToColumnar -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] +(24) RowToColumnar +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -(23) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] -Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(25) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometSort -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] -Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6], [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST] +(26) CometSort +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6], [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST] -(25) CometProject -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17, i_item_id#6] -Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +(27) CometProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -(26) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +(28) ColumnarToRow [codegen id : 7] +Input [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * ColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.date_dim (27) +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#18] +Output [2]: [d_date_sk#11, d_date#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(30) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(29) CometProject -Input [2]: [d_date_sk#11, d_date#18] +(31) CometProject +Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) ColumnarToRow [codegen id : 1] +(32) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(31) BroadcastExchange +(33) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index 2e9931a8ea..7f44f1fdd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -1,46 +1,50 @@ -WholeStageCodegen (6) +WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_category,i_class,i_item_id,i_item_desc,revenueratio] CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 RowToColumnar - WholeStageCodegen (5) + WholeStageCodegen (6) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (4) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometSort [i_class] CometColumnarExchange [i_class] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt index c8d7d7eaf3..a6e3b69c51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (34) -+- CometTakeOrderedAndProject (33) - +- CometHashAggregate (32) +TakeOrderedAndProject (34) ++- * HashAggregate (33) + +- * ColumnarToRow (32) +- CometColumnarExchange (31) +- RowToColumnar (30) +- * HashAggregate (29) @@ -179,15 +179,17 @@ Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#2 Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) CometHashAggregate +(32) ColumnarToRow [codegen id : 6] +Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] + +(33) HashAggregate [codegen id : 6] Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [3]: [_groupingexpression#14, sm_type#9, cc_name#11] Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] +Aggregate Attributes [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] +Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] -(33) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#25, sm_type#9, cc_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#25 ASC NULLS FIRST,sm_type#9 ASC NULLS FIRST,cc_name#11 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#25,sm_type#9,cc_name#11,30 days #26,31 - 60 days #27,61 - 90 days #28,91 - 120 days #29,>120 days #30]), 100, [substr(w_warehouse_name, 1, 20)#25 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#25, sm_type#9, cc_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] - -(34) ColumnarToRow [codegen id : 6] -Input [8]: [substr(w_warehouse_name, 1, 20)#25, sm_type#9, cc_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] +(34) TakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] +Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt index e24461c48c..fc410d94ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] +TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + WholeStageCodegen (6) + HashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [_groupingexpression,sm_type,cc_name] #1 RowToColumnar WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index b413027c7e..b5d11484a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (43) -+- CometTakeOrderedAndProject (42) - +- CometHashAggregate (41) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- * ColumnarToRow (41) +- CometColumnarExchange (40) +- RowToColumnar (39) +- * HashAggregate (38) @@ -227,17 +227,19 @@ Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purch Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometHashAggregate +(41) ColumnarToRow [codegen id : 10] +Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] + +(42) HashAggregate [codegen id : 10] Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#31] +Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] -(42) CometTakeOrderedAndProject -Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#21 ASC NULLS FIRST,cd_marital_status#22 ASC NULLS FIRST,cd_education_status#23 ASC NULLS FIRST,cd_purchase_estimate#24 ASC NULLS FIRST,cd_credit_rating#25 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[cd_gender#21,cd_marital_status#22,cd_education_status#23,cnt1#31,cd_purchase_estimate#24,cnt2#32,cd_credit_rating#25,cnt3#33,cd_dep_count#26,cnt4#34,cd_dep_employed_count#27,cnt5#35,cd_dep_college_count#28,cnt6#36]), 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] - -(43) ColumnarToRow [codegen id : 10] -Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33, cd_dep_count#26, cnt4#34, cd_dep_employed_count#27, cnt5#35, cd_dep_college_count#28, cnt6#36] +(43) TakeOrderedAndProject +Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] +Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] ===== Subqueries ===== @@ -250,18 +252,18 @@ BroadcastExchange (48) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#37, d_moy#38] +Output [3]: [d_date_sk#7, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct (45) CometFilter -Input [3]: [d_date_sk#7, d_year#37, d_moy#38] -Condition : (((((isnotnull(d_year#37) AND isnotnull(d_moy#38)) AND (d_year#37 = 2002)) AND (d_moy#38 >= 4)) AND (d_moy#38 <= 7)) AND isnotnull(d_date_sk#7)) +Input [3]: [d_date_sk#7, d_year#38, d_moy#39] +Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 4)) AND (d_moy#39 <= 7)) AND isnotnull(d_date_sk#7)) (46) CometProject -Input [3]: [d_date_sk#7, d_year#37, d_moy#38] +Input [3]: [d_date_sk#7, d_year#38, d_moy#39] Arguments: [d_date_sk#7], [d_date_sk#7] (47) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 440bda63b9..c455144c22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] +TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] + WholeStageCodegen (10) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] + ColumnarToRow + InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 RowToColumnar WholeStageCodegen (9) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index 0052afc39e..1428a3ae34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -5,9 +5,9 @@ TakeOrderedAndProject (79) :- * Project (58) : +- * BroadcastHashJoin Inner BuildRight (57) : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometHashAggregate (16) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * ColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- RowToColumnar (14) : : : +- * HashAggregate (13) @@ -24,8 +24,8 @@ TakeOrderedAndProject (79) : : : : +- CometScan parquet spark_catalog.default.store_sales (4) : : : +- ReusedExchange (10) : : +- BroadcastExchange (36) - : : +- * ColumnarToRow (35) - : : +- CometHashAggregate (34) + : : +- * HashAggregate (35) + : : +- * ColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- RowToColumnar (32) : : +- * HashAggregate (31) @@ -42,9 +42,9 @@ TakeOrderedAndProject (79) : : : +- CometScan parquet spark_catalog.default.store_sales (22) : : +- ReusedExchange (28) : +- BroadcastExchange (56) - : +- * ColumnarToRow (55) - : +- CometFilter (54) - : +- CometHashAggregate (53) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- * ColumnarToRow (53) : +- CometColumnarExchange (52) : +- RowToColumnar (51) : +- * HashAggregate (50) @@ -61,8 +61,8 @@ TakeOrderedAndProject (79) : : +- CometScan parquet spark_catalog.default.web_sales (41) : +- ReusedExchange (47) +- BroadcastExchange (76) - +- * ColumnarToRow (75) - +- CometHashAggregate (74) + +- * HashAggregate (75) + +- * ColumnarToRow (74) +- CometColumnarExchange (73) +- RowToColumnar (72) +- * HashAggregate (71) @@ -150,297 +150,305 @@ Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferr Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 16] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] + +(17) HashAggregate [codegen id : 16] Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] +Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] -(17) CometFilter -Input [2]: [customer_id#18, year_total#19] -Condition : (isnotnull(year_total#19) AND (year_total#19 > 0.00)) - -(18) ColumnarToRow [codegen id : 16] -Input [2]: [customer_id#18, year_total#19] +(18) Filter [codegen id : 16] +Input [2]: [customer_id#19, year_total#20] +Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Output [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (20) CometFilter -Input [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] -Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_customer_id#21)) +Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#33)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (23) CometFilter -Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -Condition : isnotnull(ss_customer_sk#28) +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Condition : isnotnull(ss_customer_sk#29) (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] (25) BroadcastExchange -Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#20] -Right keys [1]: [ss_customer_sk#28] +Left keys [1]: [c_customer_sk#21] +Right keys [1]: [ss_customer_sk#29] Join type: Inner Join condition: None (27) Project [codegen id : 6] -Output [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -Input [12]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] (28) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#33, d_year#34] +Output [2]: [d_date_sk#34, d_year#35] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#31] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [ss_sold_date_sk#32] +Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None (30) Project [codegen id : 6] -Output [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] -Input [12]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31, d_date_sk#33, d_year#34] +Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] +Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] (31) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] -Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] -Aggregate Attributes [1]: [sum#35] -Results [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] +Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] +Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] +Aggregate Attributes [1]: [sum#36] +Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] (32) RowToColumnar -Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] (33) CometColumnarExchange -Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] -Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] +Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometHashAggregate -Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#36] -Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] +(34) ColumnarToRow [codegen id : 7] +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -(35) ColumnarToRow [codegen id : 7] -Input [5]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40, year_total#41] +(35) HashAggregate [codegen id : 7] +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] +Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] +Results [5]: [c_customer_id#22 AS customer_id#38, c_first_name#23 AS customer_first_name#39, c_last_name#24 AS customer_last_name#40, c_email_address#28 AS customer_email_address#41, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#42] (36) BroadcastExchange -Input [5]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40, year_total#41] +Input [5]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] (37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#37] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#38] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49] +Output [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (39) CometFilter -Input [8]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49] -Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_customer_id#43)) +Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] +Condition : (isnotnull(c_customer_sk#43) AND isnotnull(c_customer_id#44)) (40) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49] +Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] +Output [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (42) CometFilter -Input [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_bill_customer_sk#50) +Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Condition : isnotnull(ws_bill_customer_sk#51) (43) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] +Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] (44) BroadcastExchange -Input [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] +Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] (45) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#42] -Right keys [1]: [ws_bill_customer_sk#50] +Left keys [1]: [c_customer_sk#43] +Right keys [1]: [ws_bill_customer_sk#51] Join type: Inner Join condition: None (46) Project [codegen id : 10] -Output [10]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] -Input [12]: [c_customer_sk#42, c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] +Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Input [12]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] (47) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#55, d_year#56] +Output [2]: [d_date_sk#56, d_year#57] (48) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#53] -Right keys [1]: [d_date_sk#55] +Left keys [1]: [ws_sold_date_sk#54] +Right keys [1]: [d_date_sk#56] Join type: Inner Join condition: None (49) Project [codegen id : 10] -Output [10]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, ws_ext_discount_amt#51, ws_ext_list_price#52, d_year#56] -Input [12]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] +Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] +Input [12]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] (50) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, ws_ext_discount_amt#51, ws_ext_list_price#52, d_year#56] -Keys [8]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#52 - ws_ext_discount_amt#51)))] -Aggregate Attributes [1]: [sum#57] -Results [9]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56, sum#58] +Input [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] +Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] +Aggregate Attributes [1]: [sum#58] +Results [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] (51) RowToColumnar -Input [9]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56, sum#58] +Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] (52) CometColumnarExchange -Input [9]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56, sum#58] -Arguments: hashpartitioning(c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] +Arguments: hashpartitioning(c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometHashAggregate -Input [9]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56, sum#58] -Keys [8]: [c_customer_id#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_country#47, c_login#48, c_email_address#49, d_year#56] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#52 - ws_ext_discount_amt#51)))] +(53) ColumnarToRow [codegen id : 11] +Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] -(54) CometFilter -Input [2]: [customer_id#59, year_total#60] -Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.00)) +(54) HashAggregate [codegen id : 11] +Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] +Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60] +Results [2]: [c_customer_id#44 AS customer_id#61, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60,18,2) AS year_total#62] -(55) ColumnarToRow [codegen id : 11] -Input [2]: [customer_id#59, year_total#60] +(55) Filter [codegen id : 11] +Input [2]: [customer_id#61, year_total#62] +Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.00)) (56) BroadcastExchange -Input [2]: [customer_id#59, year_total#60] +Input [2]: [customer_id#61, year_total#62] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] (57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#59] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#61] Join type: Inner Join condition: None (58) Project [codegen id : 16] -Output [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40, year_total#41, year_total#60] -Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40, year_total#41, customer_id#59, year_total#60] +Output [8]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62] +Input [9]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, customer_id#61, year_total#62] (unknown) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] +Output [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (60) CometFilter -Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] -Condition : (isnotnull(c_customer_sk#61) AND isnotnull(c_customer_id#62)) +Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] +Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_customer_id#64)) (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] +Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Output [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (63) CometFilter -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_bill_customer_sk#69) +Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +Condition : isnotnull(ws_bill_customer_sk#71) (64) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] (65) BroadcastExchange -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] (66) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#61] -Right keys [1]: [ws_bill_customer_sk#69] +Left keys [1]: [c_customer_sk#63] +Right keys [1]: [ws_bill_customer_sk#71] Join type: Inner Join condition: None (67) Project [codegen id : 14] -Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -Input [12]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Output [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +Input [12]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] (68) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#74, d_year#75] +Output [2]: [d_date_sk#76, d_year#77] (69) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#74] +Left keys [1]: [ws_sold_date_sk#74] +Right keys [1]: [d_date_sk#76] Join type: Inner Join condition: None (70) Project [codegen id : 14] -Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] -Input [12]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72, d_date_sk#74, d_year#75] +Output [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] +Input [12]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74, d_date_sk#76, d_year#77] (71) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] -Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] -Aggregate Attributes [1]: [sum#76] -Results [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] +Input [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] +Keys [8]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))] +Aggregate Attributes [1]: [sum#78] +Results [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] (72) RowToColumnar -Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] +Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] (73) CometColumnarExchange -Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] -Arguments: hashpartitioning(c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] +Arguments: hashpartitioning(c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometHashAggregate -Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] -Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] +(74) ColumnarToRow [codegen id : 15] +Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] -(75) ColumnarToRow [codegen id : 15] -Input [2]: [customer_id#78, year_total#79] +(75) HashAggregate [codegen id : 15] +Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] +Keys [8]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))#60] +Results [2]: [c_customer_id#64 AS customer_id#80, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))#60,18,2) AS year_total#81] (76) BroadcastExchange -Input [2]: [customer_id#78, year_total#79] +Input [2]: [customer_id#80, year_total#81] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] (77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#78] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#80] Join type: Inner -Join condition: (CASE WHEN (year_total#60 > 0.00) THEN (year_total#79 / year_total#60) ELSE 0E-20 END > CASE WHEN (year_total#19 > 0.00) THEN (year_total#41 / year_total#19) ELSE 0E-20 END) +Join condition: (CASE WHEN (year_total#62 > 0.00) THEN (year_total#81 / year_total#62) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END) (78) Project [codegen id : 16] -Output [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40] -Input [10]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40, year_total#41, year_total#60, customer_id#78, year_total#79] +Output [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] +Input [10]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62, customer_id#80, year_total#81] (79) TakeOrderedAndProject -Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40] -Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_first_name#38 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST, customer_email_address#40 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39, customer_email_address#40] +Input [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] +Arguments: 100, [customer_id#38 ASC NULLS FIRST, customer_first_name#39 ASC NULLS FIRST, customer_last_name#40 ASC NULLS FIRST, customer_email_address#41 ASC NULLS FIRST], [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] ===== Subqueries ===== @@ -469,7 +477,7 @@ Input [2]: [d_date_sk#14, d_year#15] Input [2]: [d_date_sk#14, d_year#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 BroadcastExchange (87) +- * ColumnarToRow (86) +- CometFilter (85) @@ -477,25 +485,25 @@ BroadcastExchange (87) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] +Output [2]: [d_date_sk#34, d_year#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (85) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) +Input [2]: [d_date_sk#34, d_year#35] +Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) (86) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#33, d_year#34] +Input [2]: [d_date_sk#34, d_year#35] (87) BroadcastExchange -Input [2]: [d_date_sk#33, d_year#34] +Input [2]: [d_date_sk#34, d_year#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#32 +Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#33 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index 6e42d76f1c..44058edfa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -5,10 +5,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] - ColumnarToRow - InputAdapter - CometFilter [year_total] - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 RowToColumnar WholeStageCodegen (3) @@ -40,9 +40,9 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 RowToColumnar WholeStageCodegen (6) @@ -74,10 +74,10 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [year_total] - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 RowToColumnar WholeStageCodegen (10) @@ -103,9 +103,9 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 RowToColumnar WholeStageCodegen (14) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index c60f2045af..0cdccc9b68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -1,26 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * Project (21) - +- Window (20) - +- * ColumnarToRow (19) - +- CometSort (18) - +- CometColumnarExchange (17) - +- CometHashAggregate (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * ColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- RowToColumnar (18) + +- * HashAggregate (17) + +- * ColumnarToRow (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.web_sales @@ -66,7 +68,7 @@ Join condition: None Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 27] +(10) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -93,63 +95,71 @@ Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] +Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16] + +(18) RowToColumnar +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -(17) CometColumnarExchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +(19) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) CometSort -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] -Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] +(20) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 4] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +(21) ColumnarToRow [codegen id : 5] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -(20) Window -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] -Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] +(22) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(21) Project [codegen id : 5] -Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] -Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] +(23) Project [codegen id : 6] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] -(22) TakeOrderedAndProject -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +(24) TakeOrderedAndProject +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#18] +Output [2]: [d_date_sk#11, d_date#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [d_date_sk#11, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(26) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(25) CometProject -Input [2]: [d_date_sk#11, d_date#18] +(27) CometProject +Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 03091f0fbb..2fb0bc9aa0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -1,40 +1,44 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (5) + WholeStageCodegen (6) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (4) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometSort [i_class] CometColumnarExchange [i_class] #1 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 377f8ee690..643f8b02d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == TakeOrderedAndProject (90) +- * BroadcastHashJoin Inner BuildRight (89) - :- * ColumnarToRow (70) - : +- CometFilter (69) - : +- CometHashAggregate (68) + :- * Filter (70) + : +- * HashAggregate (69) + : +- * ColumnarToRow (68) : +- CometColumnarExchange (67) : +- RowToColumnar (66) : +- * HashAggregate (65) @@ -72,9 +72,9 @@ TakeOrderedAndProject (90) : : +- ReusedExchange (57) : +- ReusedExchange (62) +- BroadcastExchange (88) - +- * ColumnarToRow (87) - +- CometFilter (86) - +- CometHashAggregate (85) + +- * Filter (87) + +- * HashAggregate (86) + +- * ColumnarToRow (85) +- CometColumnarExchange (84) +- RowToColumnar (83) +- * HashAggregate (82) @@ -403,113 +403,117 @@ Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) CometHashAggregate +(68) ColumnarToRow [codegen id : 52] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(69) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(69) CometFilter -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(70) ColumnarToRow [codegen id : 52] -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] +(70) Filter [codegen id : 52] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] +Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_sold_date_sk#57 IN dynamicpruning#58)] +PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (72) CometFilter -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Condition : isnotnull(ss_item_sk#54) +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Condition : isnotnull(ss_item_sk#56) (73) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] (74) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] (75) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] +Left keys [1]: [ss_item_sk#56] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None (76) ReusedExchange [Reuses operator id: 59] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] (77) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#59] +Left keys [1]: [ss_item_sk#56] +Right keys [1]: [i_item_sk#61] Join type: Inner Join condition: None (78) Project [codegen id : 50] -Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] +Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] (79) ReusedExchange [Reuses operator id: 130] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#65] (80) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#57] -Right keys [1]: [d_date_sk#63] +Left keys [1]: [ss_sold_date_sk#59] +Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None (81) Project [codegen id : 50] -Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] +Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] +Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] (82) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] +Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] +Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] +Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] (83) RowToColumnar -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] (84) CometColumnarExchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(85) CometHashAggregate -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] +(85) ColumnarToRow [codegen id : 51] +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -(86) CometFilter -Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] -Condition : (isnotnull(sales#71) AND (cast(sales#71 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) +(86) HashAggregate [codegen id : 51] +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72, count(1)#73] +Results [6]: [store AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] -(87) ColumnarToRow [codegen id : 51] -Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +(87) Filter [codegen id : 51] +Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] +Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) (88) BroadcastExchange -Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] (89) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Right keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Join type: Inner Join condition: None (90) TakeOrderedAndProject -Input [12]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51, channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] -Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51, channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] +Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] +Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] ===== Subqueries ===== -Subquery:1 Hosting operator id = 69 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* ColumnarToRow (111) -+- CometHashAggregate (110) +Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (111) ++- * ColumnarToRow (110) +- CometColumnarExchange (109) +- RowToColumnar (108) +- * HashAggregate (107) @@ -532,103 +536,105 @@ Subquery:1 Hosting operator id = 69 Hosting Expression = Subquery scalar-subquer (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75] +Output [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_sold_date_sk#75 IN dynamicpruning#76)] +PartitionFilters: [isnotnull(ss_sold_date_sk#79), dynamicpruningexpression(ss_sold_date_sk#79 IN dynamicpruning#80)] ReadSchema: struct (92) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75] +Input [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] (93) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#77] +Output [1]: [d_date_sk#81] (94) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#75] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [ss_sold_date_sk#79] +Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None (95) Project [codegen id : 2] -Output [2]: [ss_quantity#73 AS quantity#78, ss_list_price#74 AS list_price#79] -Input [4]: [ss_quantity#73, ss_list_price#74, ss_sold_date_sk#75, d_date_sk#77] +Output [2]: [ss_quantity#77 AS quantity#82, ss_list_price#78 AS list_price#83] +Input [4]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79, d_date_sk#81] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82] +Output [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#82), dynamicpruningexpression(cs_sold_date_sk#82 IN dynamicpruning#83)] +PartitionFilters: [isnotnull(cs_sold_date_sk#86), dynamicpruningexpression(cs_sold_date_sk#86 IN dynamicpruning#87)] ReadSchema: struct (97) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82] +Input [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] (98) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#84] +Output [1]: [d_date_sk#88] (99) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#82] -Right keys [1]: [d_date_sk#84] +Left keys [1]: [cs_sold_date_sk#86] +Right keys [1]: [d_date_sk#88] Join type: Inner Join condition: None (100) Project [codegen id : 4] -Output [2]: [cs_quantity#80 AS quantity#85, cs_list_price#81 AS list_price#86] -Input [4]: [cs_quantity#80, cs_list_price#81, cs_sold_date_sk#82, d_date_sk#84] +Output [2]: [cs_quantity#84 AS quantity#89, cs_list_price#85 AS list_price#90] +Input [4]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86, d_date_sk#88] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] +Output [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#90)] +PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] ReadSchema: struct (102) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] +Input [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] (103) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#91] +Output [1]: [d_date_sk#95] (104) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#91] +Left keys [1]: [ws_sold_date_sk#93] +Right keys [1]: [d_date_sk#95] Join type: Inner Join condition: None (105) Project [codegen id : 6] -Output [2]: [ws_quantity#87 AS quantity#92, ws_list_price#88 AS list_price#93] -Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#91] +Output [2]: [ws_quantity#91 AS quantity#96, ws_list_price#92 AS list_price#97] +Input [4]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93, d_date_sk#95] (106) Union (107) HashAggregate [codegen id : 7] -Input [2]: [quantity#78, list_price#79] +Input [2]: [quantity#82, list_price#83] Keys: [] -Functions [1]: [partial_avg((cast(quantity#78 as decimal(10,0)) * list_price#79))] -Aggregate Attributes [2]: [sum#94, count#95] -Results [2]: [sum#96, count#97] +Functions [1]: [partial_avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] +Aggregate Attributes [2]: [sum#98, count#99] +Results [2]: [sum#100, count#101] (108) RowToColumnar -Input [2]: [sum#96, count#97] +Input [2]: [sum#100, count#101] (109) CometColumnarExchange -Input [2]: [sum#96, count#97] +Input [2]: [sum#100, count#101] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(110) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#78 as decimal(10,0)) * list_price#79))] +(110) ColumnarToRow [codegen id : 8] +Input [2]: [sum#100, count#101] -(111) ColumnarToRow [codegen id : 8] -Input [1]: [average_sales#98] +(111) HashAggregate [codegen id : 8] +Input [2]: [sum#100, count#101] +Keys: [] +Functions [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] +Aggregate Attributes [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102] +Results [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102 AS average_sales#103] -Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#75 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#79 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#82 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (116) @@ -639,18 +645,18 @@ BroadcastExchange (116) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#42, d_week_seq#99] +Output [2]: [d_date_sk#42, d_week_seq#104] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (113) CometFilter -Input [2]: [d_date_sk#42, d_week_seq#99] -Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#42)) +Input [2]: [d_date_sk#42, d_week_seq#104] +Condition : ((isnotnull(d_week_seq#104) AND (d_week_seq#104 = Subquery scalar-subquery#105, [id=#106])) AND isnotnull(d_date_sk#42)) (114) CometProject -Input [2]: [d_date_sk#42, d_week_seq#99] +Input [2]: [d_date_sk#42, d_week_seq#104] Arguments: [d_date_sk#42], [d_date_sk#42] (115) ColumnarToRow [codegen id : 1] @@ -660,7 +666,7 @@ Input [1]: [d_date_sk#42] Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 113 Hosting Expression = Subquery scalar-subquery#100, [id=#101] +Subquery:6 Hosting operator id = 113 Hosting Expression = Subquery scalar-subquery#105, [id=#106] * ColumnarToRow (120) +- CometProject (119) +- CometFilter (118) @@ -668,22 +674,22 @@ Subquery:6 Hosting operator id = 113 Hosting Expression = Subquery scalar-subque (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] +Output [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (118) CometFilter -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 1999)) AND (d_moy#104 = 12)) AND (d_dom#105 = 16)) +Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] +Condition : (((((isnotnull(d_year#108) AND isnotnull(d_moy#109)) AND isnotnull(d_dom#110)) AND (d_year#108 = 1999)) AND (d_moy#109 = 12)) AND (d_dom#110 = 16)) (119) CometProject -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Arguments: [d_week_seq#102], [d_week_seq#102] +Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] +Arguments: [d_week_seq#107], [d_week_seq#107] (120) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#102] +Input [1]: [d_week_seq#107] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (125) @@ -694,18 +700,18 @@ BroadcastExchange (125) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#106] +Output [2]: [d_date_sk#25, d_year#111] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (122) CometFilter -Input [2]: [d_date_sk#25, d_year#106] -Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1998)) AND (d_year#106 <= 2000)) AND isnotnull(d_date_sk#25)) +Input [2]: [d_date_sk#25, d_year#111] +Condition : (((isnotnull(d_year#111) AND (d_year#111 >= 1998)) AND (d_year#111 <= 2000)) AND isnotnull(d_date_sk#25)) (123) CometProject -Input [2]: [d_date_sk#25, d_year#106] +Input [2]: [d_date_sk#25, d_year#111] Arguments: [d_date_sk#25], [d_date_sk#25] (124) ColumnarToRow [codegen id : 1] @@ -719,9 +725,9 @@ Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN d Subquery:9 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 86 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:10 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 +Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 BroadcastExchange (130) +- * ColumnarToRow (129) +- CometProject (128) @@ -730,28 +736,28 @@ BroadcastExchange (130) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#63, d_week_seq#107] +Output [2]: [d_date_sk#65, d_week_seq#112] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (127) CometFilter -Input [2]: [d_date_sk#63, d_week_seq#107] -Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#63)) +Input [2]: [d_date_sk#65, d_week_seq#112] +Condition : ((isnotnull(d_week_seq#112) AND (d_week_seq#112 = Subquery scalar-subquery#113, [id=#114])) AND isnotnull(d_date_sk#65)) (128) CometProject -Input [2]: [d_date_sk#63, d_week_seq#107] -Arguments: [d_date_sk#63], [d_date_sk#63] +Input [2]: [d_date_sk#65, d_week_seq#112] +Arguments: [d_date_sk#65], [d_date_sk#65] (129) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#63] +Input [1]: [d_date_sk#65] (130) BroadcastExchange -Input [1]: [d_date_sk#63] +Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:12 Hosting operator id = 127 Hosting Expression = Subquery scalar-subquery#108, [id=#109] +Subquery:12 Hosting operator id = 127 Hosting Expression = Subquery scalar-subquery#113, [id=#114] * ColumnarToRow (134) +- CometProject (133) +- CometFilter (132) @@ -759,21 +765,21 @@ Subquery:12 Hosting operator id = 127 Hosting Expression = Subquery scalar-subqu (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] +Output [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (132) CometFilter -Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] -Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1998)) AND (d_moy#112 = 12)) AND (d_dom#113 = 16)) +Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] +Condition : (((((isnotnull(d_year#116) AND isnotnull(d_moy#117)) AND isnotnull(d_dom#118)) AND (d_year#116 = 1998)) AND (d_moy#117 = 12)) AND (d_dom#118 = 16)) (133) CometProject -Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] -Arguments: [d_week_seq#110], [d_week_seq#110] +Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] +Arguments: [d_week_seq#115], [d_week_seq#115] (134) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#110] +Input [1]: [d_week_seq#115] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 13ac319fbd..29d5a8d87c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -1,48 +1,48 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (52) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [sales] - Subquery #4 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count] - CometColumnarExchange #12 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + Filter [sales] + Subquery #4 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + ColumnarToRow + InputAdapter + CometColumnarExchange #12 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 RowToColumnar WholeStageCodegen (25) @@ -169,11 +169,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #13 WholeStageCodegen (51) - ColumnarToRow - InputAdapter - CometFilter [sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + Filter [sales] + ReusedSubquery [average_sales] #4 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 RowToColumnar WholeStageCodegen (50) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index 0085b3b4c6..e042cdb878 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -1,135 +1,153 @@ == Physical Plan == -* ColumnarToRow (131) -+- CometTakeOrderedAndProject (130) - +- CometHashAggregate (129) - +- CometColumnarExchange (128) - +- CometHashAggregate (127) - +- CometUnion (126) - :- CometHashAggregate (105) - : +- CometColumnarExchange (104) - : +- CometHashAggregate (103) - : +- CometUnion (102) - : :- CometFilter (69) - : : +- CometHashAggregate (68) - : : +- CometColumnarExchange (67) - : : +- RowToColumnar (66) - : : +- * HashAggregate (65) - : : +- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (53) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (52) - : : : : +- * Project (51) - : : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (49) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) - : : : : :- * ColumnarToRow (37) - : : : : : +- CometHashAggregate (36) - : : : : : +- CometColumnarExchange (35) - : : : : : +- RowToColumnar (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * ColumnarToRow (9) - : : : : : : : +- CometFilter (8) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * ColumnarToRow (12) - : : : : : : : +- CometFilter (11) - : : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * ColumnarToRow (15) - : : : : : : : : +- CometFilter (14) - : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (47) - : : : : +- * Project (46) - : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : :- * Project (43) - : : : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : : : :- * ColumnarToRow (40) - : : : : : : +- CometFilter (39) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) - : : : : : +- ReusedExchange (41) - : : : : +- ReusedExchange (44) - : : : +- BroadcastExchange (59) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (58) - : : : :- * ColumnarToRow (56) - : : : : +- CometFilter (55) - : : : : +- CometScan parquet spark_catalog.default.item (54) - : : : +- ReusedExchange (57) - : : +- ReusedExchange (62) - : :- CometFilter (85) - : : +- CometHashAggregate (84) - : : +- CometColumnarExchange (83) - : : +- RowToColumnar (82) - : : +- * HashAggregate (81) - : : +- * Project (80) - : : +- * BroadcastHashJoin Inner BuildRight (79) - : : :- * Project (77) - : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (74) - : : : : :- * ColumnarToRow (72) - : : : : : +- CometFilter (71) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (70) - : : : : +- ReusedExchange (73) - : : : +- ReusedExchange (75) - : : +- ReusedExchange (78) - : +- CometFilter (101) - : +- CometHashAggregate (100) - : +- CometColumnarExchange (99) - : +- RowToColumnar (98) - : +- * HashAggregate (97) - : +- * Project (96) - : +- * BroadcastHashJoin Inner BuildRight (95) - : :- * Project (93) - : : +- * BroadcastHashJoin Inner BuildRight (92) - : : :- * BroadcastHashJoin LeftSemi BuildRight (90) - : : : :- * ColumnarToRow (88) - : : : : +- CometFilter (87) - : : : : +- CometScan parquet spark_catalog.default.web_sales (86) - : : : +- ReusedExchange (89) - : : +- ReusedExchange (91) - : +- ReusedExchange (94) - :- CometHashAggregate (110) - : +- CometColumnarExchange (109) - : +- CometHashAggregate (108) - : +- CometHashAggregate (107) - : +- ReusedExchange (106) - :- CometHashAggregate (115) - : +- CometColumnarExchange (114) - : +- CometHashAggregate (113) - : +- CometHashAggregate (112) - : +- ReusedExchange (111) - :- CometHashAggregate (120) - : +- CometColumnarExchange (119) - : +- CometHashAggregate (118) - : +- CometHashAggregate (117) - : +- ReusedExchange (116) - +- CometHashAggregate (125) - +- CometColumnarExchange (124) - +- CometHashAggregate (123) - +- CometHashAggregate (122) - +- ReusedExchange (121) +* ColumnarToRow (149) ++- CometTakeOrderedAndProject (148) + +- CometHashAggregate (147) + +- CometColumnarExchange (146) + +- RowToColumnar (145) + +- * HashAggregate (144) + +- Union (143) + :- * HashAggregate (110) + : +- * ColumnarToRow (109) + : +- CometColumnarExchange (108) + : +- RowToColumnar (107) + : +- * HashAggregate (106) + : +- Union (105) + : :- * Filter (70) + : : +- * HashAggregate (69) + : : +- * ColumnarToRow (68) + : : +- CometColumnarExchange (67) + : : +- RowToColumnar (66) + : : +- * HashAggregate (65) + : : +- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (61) + : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (53) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (52) + : : : : +- * Project (51) + : : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : : +- BroadcastExchange (49) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) + : : : : :- * ColumnarToRow (37) + : : : : : +- CometHashAggregate (36) + : : : : : +- CometColumnarExchange (35) + : : : : : +- RowToColumnar (34) + : : : : : +- * HashAggregate (33) + : : : : : +- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * ColumnarToRow (9) + : : : : : : : +- CometFilter (8) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : : +- BroadcastExchange (27) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : : :- * ColumnarToRow (12) + : : : : : : : +- CometFilter (11) + : : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : : +- BroadcastExchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * ColumnarToRow (15) + : : : : : : : : +- CometFilter (14) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * ColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : : +- ReusedExchange (22) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (47) + : : : : +- * Project (46) + : : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : : :- * Project (43) + : : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : : :- * ColumnarToRow (40) + : : : : : : +- CometFilter (39) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) + : : : : : +- ReusedExchange (41) + : : : : +- ReusedExchange (44) + : : : +- BroadcastExchange (59) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (58) + : : : :- * ColumnarToRow (56) + : : : : +- CometFilter (55) + : : : : +- CometScan parquet spark_catalog.default.item (54) + : : : +- ReusedExchange (57) + : : +- ReusedExchange (62) + : :- * Filter (87) + : : +- * HashAggregate (86) + : : +- * ColumnarToRow (85) + : : +- CometColumnarExchange (84) + : : +- RowToColumnar (83) + : : +- * HashAggregate (82) + : : +- * Project (81) + : : +- * BroadcastHashJoin Inner BuildRight (80) + : : :- * Project (78) + : : : +- * BroadcastHashJoin Inner BuildRight (77) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (75) + : : : : :- * ColumnarToRow (73) + : : : : : +- CometFilter (72) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (71) + : : : : +- ReusedExchange (74) + : : : +- ReusedExchange (76) + : : +- ReusedExchange (79) + : +- * Filter (104) + : +- * HashAggregate (103) + : +- * ColumnarToRow (102) + : +- CometColumnarExchange (101) + : +- RowToColumnar (100) + : +- * HashAggregate (99) + : +- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * BroadcastHashJoin LeftSemi BuildRight (92) + : : : :- * ColumnarToRow (90) + : : : : +- CometFilter (89) + : : : : +- CometScan parquet spark_catalog.default.web_sales (88) + : : : +- ReusedExchange (91) + : : +- ReusedExchange (93) + : +- ReusedExchange (96) + :- * HashAggregate (118) + : +- * ColumnarToRow (117) + : +- CometColumnarExchange (116) + : +- RowToColumnar (115) + : +- * HashAggregate (114) + : +- * HashAggregate (113) + : +- * ColumnarToRow (112) + : +- ReusedExchange (111) + :- * HashAggregate (126) + : +- * ColumnarToRow (125) + : +- CometColumnarExchange (124) + : +- RowToColumnar (123) + : +- * HashAggregate (122) + : +- * HashAggregate (121) + : +- * ColumnarToRow (120) + : +- ReusedExchange (119) + :- * HashAggregate (134) + : +- * ColumnarToRow (133) + : +- CometColumnarExchange (132) + : +- RowToColumnar (131) + : +- * HashAggregate (130) + : +- * HashAggregate (129) + : +- * ColumnarToRow (128) + : +- ReusedExchange (127) + +- * HashAggregate (142) + +- * ColumnarToRow (141) + +- CometColumnarExchange (140) + +- RowToColumnar (139) + +- * HashAggregate (138) + +- * HashAggregate (137) + +- * ColumnarToRow (136) + +- ReusedExchange (135) (unknown) Scan parquet spark_catalog.default.store_sales @@ -233,7 +251,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 167] +(22) ReusedExchange [Reuses operator id: 185] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -270,7 +288,7 @@ Join condition: None Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 167] +(30) ReusedExchange [Reuses operator id: 185] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 6] @@ -333,7 +351,7 @@ Join condition: None Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(44) ReusedExchange [Reuses operator id: 167] +(44) ReusedExchange [Reuses operator id: 185] Output [1]: [d_date_sk#36] (45) BroadcastHashJoin [codegen id : 9] @@ -417,7 +435,7 @@ Join condition: None Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(62) ReusedExchange [Reuses operator id: 162] +(62) ReusedExchange [Reuses operator id: 180] Output [1]: [d_date_sk#42] (63) BroadcastHashJoin [codegen id : 25] @@ -444,506 +462,590 @@ Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) CometHashAggregate +(68) ColumnarToRow [codegen id : 26] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(69) HashAggregate [codegen id : 26] Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(69) CometFilter -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) +(70) Filter [codegen id : 26] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Output [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] +PartitionFilters: [isnotnull(cs_sold_date_sk#59), dynamicpruningexpression(cs_sold_date_sk#59 IN dynamicpruning#60)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(71) CometFilter -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) +(72) CometFilter +Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +Condition : isnotnull(cs_item_sk#56) -(72) ColumnarToRow [codegen id : 50] -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +(73) ColumnarToRow [codegen id : 51] +Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] -(73) ReusedExchange [Reuses operator id: 52] +(74) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(74) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [cs_item_sk#54] +(75) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#56] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(75) ReusedExchange [Reuses operator id: 59] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +(76) ReusedExchange [Reuses operator id: 59] +Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -(76) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [cs_item_sk#54] -Right keys [1]: [i_item_sk#59] +(77) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#56] +Right keys [1]: [i_item_sk#61] Join type: Inner Join condition: None -(77) Project [codegen id : 50] -Output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +(78) Project [codegen id : 51] +Output [6]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] +Input [8]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -(78) ReusedExchange [Reuses operator id: 162] -Output [1]: [d_date_sk#63] +(79) ReusedExchange [Reuses operator id: 180] +Output [1]: [d_date_sk#65] -(79) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [cs_sold_date_sk#57] -Right keys [1]: [d_date_sk#63] +(80) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_sold_date_sk#59] +Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(80) Project [codegen id : 50] -Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] +(81) Project [codegen id : 51] +Output [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] +Input [7]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -(81) HashAggregate [codegen id : 50] -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +(82) HashAggregate [codegen id : 51] +Input [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] +Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Functions [2]: [partial_sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), partial_count(1)] +Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] +Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -(82) RowToColumnar -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] +(83) RowToColumnar +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -(83) CometColumnarExchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(84) CometColumnarExchange +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(84) CometHashAggregate -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] +(85) ColumnarToRow [codegen id : 52] +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -(85) CometFilter -Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] -Condition : (isnotnull(sales#71) AND (cast(sales#71 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) +(86) HashAggregate [codegen id : 52] +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Functions [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#72, count(1)#73] +Results [6]: [catalog AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] + +(87) Filter [codegen id : 52] +Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] +Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] +Output [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#81)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(87) CometFilter -Input [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] -Condition : isnotnull(ws_item_sk#73) +(89) CometFilter +Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] +Condition : isnotnull(ws_item_sk#77) -(88) ColumnarToRow [codegen id : 75] -Input [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] +(90) ColumnarToRow [codegen id : 77] +Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] -(89) ReusedExchange [Reuses operator id: 52] +(91) ReusedExchange [Reuses operator id: 52] Output [1]: [ss_item_sk#37] -(90) BroadcastHashJoin [codegen id : 75] -Left keys [1]: [ws_item_sk#73] +(92) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#77] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(91) ReusedExchange [Reuses operator id: 59] -Output [4]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] +(93) ReusedExchange [Reuses operator id: 59] +Output [4]: [i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] -(92) BroadcastHashJoin [codegen id : 75] -Left keys [1]: [ws_item_sk#73] -Right keys [1]: [i_item_sk#78] +(94) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#77] +Right keys [1]: [i_item_sk#82] Join type: Inner Join condition: None -(93) Project [codegen id : 75] -Output [6]: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81] -Input [8]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] +(95) Project [codegen id : 77] +Output [6]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#83, i_class_id#84, i_category_id#85] +Input [8]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] -(94) ReusedExchange [Reuses operator id: 162] -Output [1]: [d_date_sk#82] +(96) ReusedExchange [Reuses operator id: 180] +Output [1]: [d_date_sk#86] -(95) BroadcastHashJoin [codegen id : 75] -Left keys [1]: [ws_sold_date_sk#76] -Right keys [1]: [d_date_sk#82] +(97) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_sold_date_sk#80] +Right keys [1]: [d_date_sk#86] Join type: Inner Join condition: None -(96) Project [codegen id : 75] -Output [5]: [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81] -Input [7]: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, d_date_sk#82] - -(97) HashAggregate [codegen id : 75] -Input [5]: [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81] -Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] -Functions [2]: [partial_sum((cast(ws_quantity#74 as decimal(10,0)) * ws_list_price#75)), partial_count(1)] -Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] -Results [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] - -(98) RowToColumnar -Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] - -(99) CometColumnarExchange -Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] -Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(100) CometHashAggregate -Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] -Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] -Functions [2]: [sum((cast(ws_quantity#74 as decimal(10,0)) * ws_list_price#75)), count(1)] - -(101) CometFilter -Input [6]: [channel#89, i_brand_id#79, i_class_id#80, i_category_id#81, sales#90, number_sales#91] -Condition : (isnotnull(sales#90) AND (cast(sales#90 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(102) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] -Child 1 Input [6]: [channel#70, i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72] -Child 2 Input [6]: [channel#89, i_brand_id#79, i_class_id#80, i_category_id#81, sales#90, number_sales#91] - -(103) CometHashAggregate -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] - -(104) CometColumnarExchange -Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] -Arguments: hashpartitioning(channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(105) CometHashAggregate -Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] -Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(106) ReusedExchange [Reuses operator id: 104] -Output [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] - -(107) CometHashAggregate -Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] -Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(108) CometHashAggregate -Input [5]: [channel#49, i_brand_id#39, i_class_id#40, sum_sales#95, number_sales#96] -Keys [3]: [channel#49, i_brand_id#39, i_class_id#40] -Functions [2]: [partial_sum(sum_sales#95), partial_sum(number_sales#96)] - -(109) CometColumnarExchange -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, sum#97, isEmpty#98, sum#99] -Arguments: hashpartitioning(channel#49, i_brand_id#39, i_class_id#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(110) CometHashAggregate -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, sum#97, isEmpty#98, sum#99] -Keys [3]: [channel#49, i_brand_id#39, i_class_id#40] -Functions [2]: [sum(sum_sales#95), sum(number_sales#96)] - -(111) ReusedExchange [Reuses operator id: 104] -Output [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] - -(112) CometHashAggregate -Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] -Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(113) CometHashAggregate -Input [4]: [channel#49, i_brand_id#39, sum_sales#95, number_sales#96] -Keys [2]: [channel#49, i_brand_id#39] -Functions [2]: [partial_sum(sum_sales#95), partial_sum(number_sales#96)] - -(114) CometColumnarExchange -Input [5]: [channel#49, i_brand_id#39, sum#100, isEmpty#101, sum#102] -Arguments: hashpartitioning(channel#49, i_brand_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(115) CometHashAggregate -Input [5]: [channel#49, i_brand_id#39, sum#100, isEmpty#101, sum#102] -Keys [2]: [channel#49, i_brand_id#39] -Functions [2]: [sum(sum_sales#95), sum(number_sales#96)] - -(116) ReusedExchange [Reuses operator id: 104] -Output [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] - -(117) CometHashAggregate -Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] -Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(118) CometHashAggregate -Input [3]: [channel#49, sum_sales#95, number_sales#96] -Keys [1]: [channel#49] -Functions [2]: [partial_sum(sum_sales#95), partial_sum(number_sales#96)] - -(119) CometColumnarExchange -Input [4]: [channel#49, sum#103, isEmpty#104, sum#105] -Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(120) CometHashAggregate -Input [4]: [channel#49, sum#103, isEmpty#104, sum#105] -Keys [1]: [channel#49] -Functions [2]: [sum(sum_sales#95), sum(number_sales#96)] - -(121) ReusedExchange [Reuses operator id: 104] -Output [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] - -(122) CometHashAggregate -Input [7]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum#92, isEmpty#93, sum#94] -Keys [4]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(123) CometHashAggregate -Input [2]: [sum_sales#95, number_sales#96] -Keys: [] -Functions [2]: [partial_sum(sum_sales#95), partial_sum(number_sales#96)] +(98) Project [codegen id : 77] +Output [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] +Input [7]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#83, i_class_id#84, i_category_id#85, d_date_sk#86] + +(99) HashAggregate [codegen id : 77] +Input [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] +Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] +Functions [2]: [partial_sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), partial_count(1)] +Aggregate Attributes [3]: [sum#87, isEmpty#88, count#89] +Results [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] + +(100) RowToColumnar +Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] + +(101) CometColumnarExchange +Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] +Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(102) ColumnarToRow [codegen id : 78] +Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] + +(103) HashAggregate [codegen id : 78] +Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] +Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] +Functions [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#93, count(1)#94] +Results [6]: [web AS channel#95, i_brand_id#83, i_class_id#84, i_category_id#85, sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#93 AS sales#96, count(1)#94 AS number_sales#97] + +(104) Filter [codegen id : 78] +Input [6]: [channel#95, i_brand_id#83, i_class_id#84, i_category_id#85, sales#96, number_sales#97] +Condition : (isnotnull(sales#96) AND (cast(sales#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(105) Union + +(106) HashAggregate [codegen id : 79] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] +Aggregate Attributes [3]: [sum#98, isEmpty#99, sum#100] +Results [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] + +(107) RowToColumnar +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] + +(108) CometColumnarExchange +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(109) ColumnarToRow [codegen id : 80] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] + +(110) HashAggregate [codegen id : 80] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] + +(111) ReusedExchange [Reuses operator id: 108] +Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] + +(112) ColumnarToRow [codegen id : 160] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] + +(113) HashAggregate [codegen id : 160] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] +Results [5]: [channel#51, i_brand_id#39, i_class_id#40, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] + +(114) HashAggregate [codegen id : 160] +Input [5]: [channel#51, i_brand_id#39, i_class_id#40, sum_sales#106, number_sales#107] +Keys [3]: [channel#51, i_brand_id#39, i_class_id#40] +Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] +Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] + +(115) RowToColumnar +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] + +(116) CometColumnarExchange +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(117) ColumnarToRow [codegen id : 161] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] + +(118) HashAggregate [codegen id : 161] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] +Keys [3]: [channel#51, i_brand_id#39, i_class_id#40] +Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] +Aggregate Attributes [2]: [sum(sum_sales#106)#114, sum(number_sales#107)#115] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, null AS i_category_id#116, sum(sum_sales#106)#114 AS sum(sum_sales)#117, sum(number_sales#107)#115 AS sum(number_sales)#118] + +(119) ReusedExchange [Reuses operator id: 108] +Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] + +(120) ColumnarToRow [codegen id : 241] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] + +(121) HashAggregate [codegen id : 241] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] +Results [4]: [channel#51, i_brand_id#39, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] + +(122) HashAggregate [codegen id : 241] +Input [4]: [channel#51, i_brand_id#39, sum_sales#106, number_sales#107] +Keys [2]: [channel#51, i_brand_id#39] +Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] +Aggregate Attributes [3]: [sum#119, isEmpty#120, sum#121] +Results [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] + +(123) RowToColumnar +Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] (124) CometColumnarExchange -Input [3]: [sum#106, isEmpty#107, sum#108] +Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] +Arguments: hashpartitioning(channel#51, i_brand_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(125) ColumnarToRow [codegen id : 242] +Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] + +(126) HashAggregate [codegen id : 242] +Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] +Keys [2]: [channel#51, i_brand_id#39] +Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] +Aggregate Attributes [2]: [sum(sum_sales#106)#125, sum(number_sales#107)#126] +Results [6]: [channel#51, i_brand_id#39, null AS i_class_id#127, null AS i_category_id#128, sum(sum_sales#106)#125 AS sum(sum_sales)#129, sum(number_sales#107)#126 AS sum(number_sales)#130] + +(127) ReusedExchange [Reuses operator id: 108] +Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] + +(128) ColumnarToRow [codegen id : 322] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] + +(129) HashAggregate [codegen id : 322] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] +Results [3]: [channel#51, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] + +(130) HashAggregate [codegen id : 322] +Input [3]: [channel#51, sum_sales#106, number_sales#107] +Keys [1]: [channel#51] +Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] +Aggregate Attributes [3]: [sum#131, isEmpty#132, sum#133] +Results [4]: [channel#51, sum#134, isEmpty#135, sum#136] + +(131) RowToColumnar +Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] + +(132) CometColumnarExchange +Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] +Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] + +(133) ColumnarToRow [codegen id : 323] +Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] + +(134) HashAggregate [codegen id : 323] +Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] +Keys [1]: [channel#51] +Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] +Aggregate Attributes [2]: [sum(sum_sales#106)#137, sum(number_sales#107)#138] +Results [6]: [channel#51, null AS i_brand_id#139, null AS i_class_id#140, null AS i_category_id#141, sum(sum_sales#106)#137 AS sum(sum_sales)#142, sum(number_sales#107)#138 AS sum(number_sales)#143] + +(135) ReusedExchange [Reuses operator id: 108] +Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] + +(136) ColumnarToRow [codegen id : 403] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] + +(137) HashAggregate [codegen id : 403] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] +Results [2]: [sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] + +(138) HashAggregate [codegen id : 403] +Input [2]: [sum_sales#106, number_sales#107] +Keys: [] +Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] +Aggregate Attributes [3]: [sum#144, isEmpty#145, sum#146] +Results [3]: [sum#147, isEmpty#148, sum#149] + +(139) RowToColumnar +Input [3]: [sum#147, isEmpty#148, sum#149] + +(140) CometColumnarExchange +Input [3]: [sum#147, isEmpty#148, sum#149] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(125) CometHashAggregate -Input [3]: [sum#106, isEmpty#107, sum#108] +(141) ColumnarToRow [codegen id : 404] +Input [3]: [sum#147, isEmpty#148, sum#149] + +(142) HashAggregate [codegen id : 404] +Input [3]: [sum#147, isEmpty#148, sum#149] Keys: [] -Functions [2]: [sum(sum_sales#95), sum(number_sales#96)] - -(126) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] -Child 1 Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#109, sum(sum_sales)#110, sum(number_sales)#111] -Child 2 Input [6]: [channel#49, i_brand_id#39, i_class_id#112, i_category_id#113, sum(sum_sales)#114, sum(number_sales)#115] -Child 3 Input [6]: [channel#49, i_brand_id#116, i_class_id#117, i_category_id#118, sum(sum_sales)#119, sum(number_sales)#120] -Child 4 Input [6]: [channel#121, i_brand_id#122, i_class_id#123, i_category_id#124, sum(sum_sales)#125, sum(number_sales)#126] - -(127) CometHashAggregate -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] -Keys [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] +Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] +Aggregate Attributes [2]: [sum(sum_sales#106)#150, sum(number_sales#107)#151] +Results [6]: [null AS channel#152, null AS i_brand_id#153, null AS i_class_id#154, null AS i_category_id#155, sum(sum_sales#106)#150 AS sum(sum_sales)#156, sum(number_sales#107)#151 AS sum(number_sales)#157] + +(143) Union + +(144) HashAggregate [codegen id : 405] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] Functions: [] +Aggregate Attributes: [] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -(128) CometColumnarExchange -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] -Arguments: hashpartitioning(channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +(145) RowToColumnar +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -(129) CometHashAggregate -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] -Keys [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] +(146) CometColumnarExchange +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] + +(147) CometHashAggregate +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] Functions: [] -(130) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#49,i_brand_id#39,i_class_id#40,i_category_id#41,sum_sales#95,number_sales#96]), 100, [channel#49 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] +(148) CometTakeOrderedAndProject +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#51 ASC NULLS FIRST,i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#51,i_brand_id#39,i_class_id#40,i_category_id#41,sum_sales#106,number_sales#107]), 100, [channel#51 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -(131) ColumnarToRow [codegen id : 376] -Input [6]: [channel#49, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#95, number_sales#96] +(149) ColumnarToRow [codegen id : 406] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] ===== Subqueries ===== -Subquery:1 Hosting operator id = 69 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* ColumnarToRow (152) -+- CometHashAggregate (151) - +- CometColumnarExchange (150) - +- RowToColumnar (149) - +- * HashAggregate (148) - +- Union (147) - :- * Project (136) - : +- * BroadcastHashJoin Inner BuildRight (135) - : :- * ColumnarToRow (133) - : : +- CometScan parquet spark_catalog.default.store_sales (132) - : +- ReusedExchange (134) - :- * Project (141) - : +- * BroadcastHashJoin Inner BuildRight (140) - : :- * ColumnarToRow (138) - : : +- CometScan parquet spark_catalog.default.catalog_sales (137) - : +- ReusedExchange (139) - +- * Project (146) - +- * BroadcastHashJoin Inner BuildRight (145) - :- * ColumnarToRow (143) - : +- CometScan parquet spark_catalog.default.web_sales (142) - +- ReusedExchange (144) +Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (170) ++- * ColumnarToRow (169) + +- CometColumnarExchange (168) + +- RowToColumnar (167) + +- * HashAggregate (166) + +- Union (165) + :- * Project (154) + : +- * BroadcastHashJoin Inner BuildRight (153) + : :- * ColumnarToRow (151) + : : +- CometScan parquet spark_catalog.default.store_sales (150) + : +- ReusedExchange (152) + :- * Project (159) + : +- * BroadcastHashJoin Inner BuildRight (158) + : :- * ColumnarToRow (156) + : : +- CometScan parquet spark_catalog.default.catalog_sales (155) + : +- ReusedExchange (157) + +- * Project (164) + +- * BroadcastHashJoin Inner BuildRight (163) + :- * ColumnarToRow (161) + : +- CometScan parquet spark_catalog.default.web_sales (160) + +- ReusedExchange (162) (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#127, ss_list_price#128, ss_sold_date_sk#129] +Output [3]: [ss_quantity#158, ss_list_price#159, ss_sold_date_sk#160] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#129), dynamicpruningexpression(ss_sold_date_sk#129 IN dynamicpruning#130)] +PartitionFilters: [isnotnull(ss_sold_date_sk#160), dynamicpruningexpression(ss_sold_date_sk#160 IN dynamicpruning#161)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#127, ss_list_price#128, ss_sold_date_sk#129] +(151) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#158, ss_list_price#159, ss_sold_date_sk#160] -(134) ReusedExchange [Reuses operator id: 167] -Output [1]: [d_date_sk#131] +(152) ReusedExchange [Reuses operator id: 185] +Output [1]: [d_date_sk#162] -(135) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#129] -Right keys [1]: [d_date_sk#131] +(153) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#160] +Right keys [1]: [d_date_sk#162] Join type: Inner Join condition: None -(136) Project [codegen id : 2] -Output [2]: [ss_quantity#127 AS quantity#132, ss_list_price#128 AS list_price#133] -Input [4]: [ss_quantity#127, ss_list_price#128, ss_sold_date_sk#129, d_date_sk#131] +(154) Project [codegen id : 2] +Output [2]: [ss_quantity#158 AS quantity#163, ss_list_price#159 AS list_price#164] +Input [4]: [ss_quantity#158, ss_list_price#159, ss_sold_date_sk#160, d_date_sk#162] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#134, cs_list_price#135, cs_sold_date_sk#136] +Output [3]: [cs_quantity#165, cs_list_price#166, cs_sold_date_sk#167] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#136), dynamicpruningexpression(cs_sold_date_sk#136 IN dynamicpruning#137)] +PartitionFilters: [isnotnull(cs_sold_date_sk#167), dynamicpruningexpression(cs_sold_date_sk#167 IN dynamicpruning#168)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#134, cs_list_price#135, cs_sold_date_sk#136] +(156) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#165, cs_list_price#166, cs_sold_date_sk#167] -(139) ReusedExchange [Reuses operator id: 157] -Output [1]: [d_date_sk#138] +(157) ReusedExchange [Reuses operator id: 175] +Output [1]: [d_date_sk#169] -(140) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#136] -Right keys [1]: [d_date_sk#138] +(158) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#167] +Right keys [1]: [d_date_sk#169] Join type: Inner Join condition: None -(141) Project [codegen id : 4] -Output [2]: [cs_quantity#134 AS quantity#139, cs_list_price#135 AS list_price#140] -Input [4]: [cs_quantity#134, cs_list_price#135, cs_sold_date_sk#136, d_date_sk#138] +(159) Project [codegen id : 4] +Output [2]: [cs_quantity#165 AS quantity#170, cs_list_price#166 AS list_price#171] +Input [4]: [cs_quantity#165, cs_list_price#166, cs_sold_date_sk#167, d_date_sk#169] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#141, ws_list_price#142, ws_sold_date_sk#143] +Output [3]: [ws_quantity#172, ws_list_price#173, ws_sold_date_sk#174] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#143), dynamicpruningexpression(ws_sold_date_sk#143 IN dynamicpruning#144)] +PartitionFilters: [isnotnull(ws_sold_date_sk#174), dynamicpruningexpression(ws_sold_date_sk#174 IN dynamicpruning#175)] ReadSchema: struct -(143) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#141, ws_list_price#142, ws_sold_date_sk#143] +(161) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#172, ws_list_price#173, ws_sold_date_sk#174] -(144) ReusedExchange [Reuses operator id: 157] -Output [1]: [d_date_sk#145] +(162) ReusedExchange [Reuses operator id: 175] +Output [1]: [d_date_sk#176] -(145) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#143] -Right keys [1]: [d_date_sk#145] +(163) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#174] +Right keys [1]: [d_date_sk#176] Join type: Inner Join condition: None -(146) Project [codegen id : 6] -Output [2]: [ws_quantity#141 AS quantity#146, ws_list_price#142 AS list_price#147] -Input [4]: [ws_quantity#141, ws_list_price#142, ws_sold_date_sk#143, d_date_sk#145] +(164) Project [codegen id : 6] +Output [2]: [ws_quantity#172 AS quantity#177, ws_list_price#173 AS list_price#178] +Input [4]: [ws_quantity#172, ws_list_price#173, ws_sold_date_sk#174, d_date_sk#176] -(147) Union +(165) Union -(148) HashAggregate [codegen id : 7] -Input [2]: [quantity#132, list_price#133] +(166) HashAggregate [codegen id : 7] +Input [2]: [quantity#163, list_price#164] Keys: [] -Functions [1]: [partial_avg((cast(quantity#132 as decimal(10,0)) * list_price#133))] -Aggregate Attributes [2]: [sum#148, count#149] -Results [2]: [sum#150, count#151] +Functions [1]: [partial_avg((cast(quantity#163 as decimal(10,0)) * list_price#164))] +Aggregate Attributes [2]: [sum#179, count#180] +Results [2]: [sum#181, count#182] -(149) RowToColumnar -Input [2]: [sum#150, count#151] +(167) RowToColumnar +Input [2]: [sum#181, count#182] -(150) CometColumnarExchange -Input [2]: [sum#150, count#151] +(168) CometColumnarExchange +Input [2]: [sum#181, count#182] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(151) CometHashAggregate -Input [2]: [sum#150, count#151] -Keys: [] -Functions [1]: [avg((cast(quantity#132 as decimal(10,0)) * list_price#133))] +(169) ColumnarToRow [codegen id : 8] +Input [2]: [sum#181, count#182] -(152) ColumnarToRow [codegen id : 8] -Input [1]: [average_sales#152] +(170) HashAggregate [codegen id : 8] +Input [2]: [sum#181, count#182] +Keys: [] +Functions [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))] +Aggregate Attributes [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))#183] +Results [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))#183 AS average_sales#184] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#129 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 150 Hosting Expression = ss_sold_date_sk#160 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 137 Hosting Expression = cs_sold_date_sk#136 IN dynamicpruning#137 -BroadcastExchange (157) -+- * ColumnarToRow (156) - +- CometProject (155) - +- CometFilter (154) - +- CometScan parquet spark_catalog.default.date_dim (153) +Subquery:3 Hosting operator id = 155 Hosting Expression = cs_sold_date_sk#167 IN dynamicpruning#168 +BroadcastExchange (175) ++- * ColumnarToRow (174) + +- CometProject (173) + +- CometFilter (172) + +- CometScan parquet spark_catalog.default.date_dim (171) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#138, d_year#153] +Output [2]: [d_date_sk#169, d_year#185] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(154) CometFilter -Input [2]: [d_date_sk#138, d_year#153] -Condition : (((isnotnull(d_year#153) AND (d_year#153 >= 1998)) AND (d_year#153 <= 2000)) AND isnotnull(d_date_sk#138)) +(172) CometFilter +Input [2]: [d_date_sk#169, d_year#185] +Condition : (((isnotnull(d_year#185) AND (d_year#185 >= 1998)) AND (d_year#185 <= 2000)) AND isnotnull(d_date_sk#169)) -(155) CometProject -Input [2]: [d_date_sk#138, d_year#153] -Arguments: [d_date_sk#138], [d_date_sk#138] +(173) CometProject +Input [2]: [d_date_sk#169, d_year#185] +Arguments: [d_date_sk#169], [d_date_sk#169] -(156) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#138] +(174) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#169] -(157) BroadcastExchange -Input [1]: [d_date_sk#138] +(175) BroadcastExchange +Input [1]: [d_date_sk#169] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -Subquery:4 Hosting operator id = 142 Hosting Expression = ws_sold_date_sk#143 IN dynamicpruning#137 +Subquery:4 Hosting operator id = 160 Hosting Expression = ws_sold_date_sk#174 IN dynamicpruning#168 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (162) -+- * ColumnarToRow (161) - +- CometProject (160) - +- CometFilter (159) - +- CometScan parquet spark_catalog.default.date_dim (158) +BroadcastExchange (180) ++- * ColumnarToRow (179) + +- CometProject (178) + +- CometFilter (177) + +- CometScan parquet spark_catalog.default.date_dim (176) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#42, d_year#154, d_moy#155] +Output [3]: [d_date_sk#42, d_year#186, d_moy#187] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(159) CometFilter -Input [3]: [d_date_sk#42, d_year#154, d_moy#155] -Condition : ((((isnotnull(d_year#154) AND isnotnull(d_moy#155)) AND (d_year#154 = 2000)) AND (d_moy#155 = 11)) AND isnotnull(d_date_sk#42)) +(177) CometFilter +Input [3]: [d_date_sk#42, d_year#186, d_moy#187] +Condition : ((((isnotnull(d_year#186) AND isnotnull(d_moy#187)) AND (d_year#186 = 2000)) AND (d_moy#187 = 11)) AND isnotnull(d_date_sk#42)) -(160) CometProject -Input [3]: [d_date_sk#42, d_year#154, d_moy#155] +(178) CometProject +Input [3]: [d_date_sk#42, d_year#186, d_moy#187] Arguments: [d_date_sk#42], [d_date_sk#42] -(161) ColumnarToRow [codegen id : 1] +(179) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#42] -(162) BroadcastExchange +(180) BroadcastExchange Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (167) -+- * ColumnarToRow (166) - +- CometProject (165) - +- CometFilter (164) - +- CometScan parquet spark_catalog.default.date_dim (163) +BroadcastExchange (185) ++- * ColumnarToRow (184) + +- CometProject (183) + +- CometFilter (182) + +- CometScan parquet spark_catalog.default.date_dim (181) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#156] +Output [2]: [d_date_sk#25, d_year#188] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(164) CometFilter -Input [2]: [d_date_sk#25, d_year#156] -Condition : (((isnotnull(d_year#156) AND (d_year#156 >= 1999)) AND (d_year#156 <= 2001)) AND isnotnull(d_date_sk#25)) +(182) CometFilter +Input [2]: [d_date_sk#25, d_year#188] +Condition : (((isnotnull(d_year#188) AND (d_year#188 >= 1999)) AND (d_year#188 <= 2001)) AND isnotnull(d_date_sk#25)) -(165) CometProject -Input [2]: [d_date_sk#25, d_year#156] +(183) CometProject +Input [2]: [d_date_sk#25, d_year#188] Arguments: [d_date_sk#25], [d_date_sk#25] -(166) ColumnarToRow [codegen id : 1] +(184) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(167) BroadcastExchange +(185) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] @@ -951,12 +1053,12 @@ Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN d Subquery:8 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:9 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:10 Hosting operator id = 70 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 71 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 101 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:11 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:12 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index becceb8614..90bff1665b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -1,241 +1,287 @@ -WholeStageCodegen (376) +WholeStageCodegen (406) ColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometUnion - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometUnion - CometFilter [sales] - Subquery #3 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count] - CometColumnarExchange #14 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 + RowToColumnar + WholeStageCodegen (405) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + InputAdapter + Union + WholeStageCodegen (80) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + RowToColumnar + WholeStageCodegen (79) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (26) + Filter [sales] + Subquery #3 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #15 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 - RowToColumnar - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - ColumnarToRow - InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #7 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarExchange #14 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #15 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #15 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #15 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 + RowToColumnar + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #7 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #4 - CometFilter [sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 - RowToColumnar - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - CometFilter [sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 - RowToColumnar - WholeStageCodegen (75) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - CometHashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] - CometColumnarExchange [channel,i_brand_id,i_class_id] #18 - CometHashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [channel,i_brand_id,sum,isEmpty,sum] - CometColumnarExchange [channel,i_brand_id] #19 - CometHashAggregate [channel,i_brand_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [channel,sum,isEmpty,sum] - CometColumnarExchange [channel] #20 - CometHashAggregate [channel,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] - CometColumnarExchange #21 - CometHashAggregate [sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (52) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + RowToColumnar + WholeStageCodegen (51) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (78) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 + RowToColumnar + WholeStageCodegen (77) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (161) + HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id] #18 + RowToColumnar + WholeStageCodegen (160) + HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + ColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (242) + HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id] #19 + RowToColumnar + WholeStageCodegen (241) + HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + ColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (323) + HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #20 + RowToColumnar + WholeStageCodegen (322) + HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + ColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (404) + HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange #21 + RowToColumnar + WholeStageCodegen (403) + HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + ColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt index 7130612b73..70d144d28c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt @@ -1,163 +1,167 @@ == Physical Plan == -* ColumnarToRow (159) -+- CometTakeOrderedAndProject (158) - +- CometUnion (157) - :- CometHashAggregate (42) - : +- CometColumnarExchange (41) - : +- RowToColumnar (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * ColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan parquet spark_catalog.default.customer (11) - : : : : +- BroadcastExchange (21) - : : : : +- * ColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) - : : : +- BroadcastExchange (27) - : : : +- * ColumnarToRow (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.customer_address (24) - : : +- ReusedExchange (30) - : +- BroadcastExchange (36) - : +- * ColumnarToRow (35) - : +- CometFilter (34) - : +- CometScan parquet spark_catalog.default.item (33) - :- CometHashAggregate (70) - : +- CometColumnarExchange (69) - : +- RowToColumnar (68) - : +- * HashAggregate (67) - : +- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * Project (54) - : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : :- * Project (51) - : : : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : : : :- * Project (48) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : :- * ColumnarToRow (45) - : : : : : : : +- CometFilter (44) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (43) - : : : : : : +- ReusedExchange (46) - : : : : : +- ReusedExchange (49) - : : : : +- ReusedExchange (52) - : : : +- BroadcastExchange (58) - : : : +- * ColumnarToRow (57) - : : : +- CometFilter (56) - : : : +- CometScan parquet spark_catalog.default.customer_address (55) - : : +- ReusedExchange (61) - : +- ReusedExchange (64) - :- CometHashAggregate (99) - : +- CometColumnarExchange (98) - : +- RowToColumnar (97) - : +- * HashAggregate (96) - : +- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (89) - : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : :- * Project (82) - : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : :- * Project (79) - : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : :- * Project (76) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : : : : :- * ColumnarToRow (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (71) - : : : : : : +- ReusedExchange (74) - : : : : : +- ReusedExchange (77) - : : : : +- ReusedExchange (80) - : : : +- BroadcastExchange (87) - : : : +- * ColumnarToRow (86) - : : : +- CometProject (85) - : : : +- CometFilter (84) - : : : +- CometScan parquet spark_catalog.default.customer_address (83) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- CometHashAggregate (128) - : +- CometColumnarExchange (127) - : +- RowToColumnar (126) - : +- * HashAggregate (125) - : +- * Project (124) - : +- * BroadcastHashJoin Inner BuildRight (123) - : :- * Project (121) - : : +- * BroadcastHashJoin Inner BuildRight (120) - : : :- * Project (118) - : : : +- * BroadcastHashJoin Inner BuildRight (117) - : : : :- * Project (111) - : : : : +- * BroadcastHashJoin Inner BuildRight (110) - : : : : :- * Project (108) - : : : : : +- * BroadcastHashJoin Inner BuildRight (107) - : : : : : :- * Project (105) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (104) - : : : : : : :- * ColumnarToRow (102) - : : : : : : : +- CometFilter (101) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (100) - : : : : : : +- ReusedExchange (103) - : : : : : +- ReusedExchange (106) - : : : : +- ReusedExchange (109) - : : : +- BroadcastExchange (116) - : : : +- * ColumnarToRow (115) - : : : +- CometProject (114) - : : : +- CometFilter (113) - : : : +- CometScan parquet spark_catalog.default.customer_address (112) - : : +- ReusedExchange (119) - : +- ReusedExchange (122) - +- CometHashAggregate (156) - +- CometColumnarExchange (155) - +- RowToColumnar (154) - +- * HashAggregate (153) - +- * Project (152) - +- * BroadcastHashJoin Inner BuildRight (151) - :- * Project (146) - : +- * BroadcastHashJoin Inner BuildRight (145) - : :- * Project (143) - : : +- * BroadcastHashJoin Inner BuildRight (142) - : : :- * Project (140) - : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : :- * Project (137) - : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : :- * Project (134) - : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : :- * ColumnarToRow (131) - : : : : : : +- CometFilter (130) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (129) - : : : : : +- ReusedExchange (132) - : : : : +- ReusedExchange (135) - : : : +- ReusedExchange (138) - : : +- ReusedExchange (141) - : +- ReusedExchange (144) - +- BroadcastExchange (150) - +- * ColumnarToRow (149) - +- CometFilter (148) - +- CometScan parquet spark_catalog.default.item (147) +TakeOrderedAndProject (163) ++- Union (162) + :- * HashAggregate (43) + : +- * ColumnarToRow (42) + : +- CometColumnarExchange (41) + : +- RowToColumnar (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (32) + : : +- * BroadcastHashJoin Inner BuildRight (31) + : : :- * Project (29) + : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : :- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * ColumnarToRow (7) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- CometProject (13) + : : : : : +- CometFilter (12) + : : : : : +- CometScan parquet spark_catalog.default.customer (11) + : : : : +- BroadcastExchange (21) + : : : : +- * ColumnarToRow (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) + : : : +- BroadcastExchange (27) + : : : +- * ColumnarToRow (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.customer_address (24) + : : +- ReusedExchange (30) + : +- BroadcastExchange (36) + : +- * ColumnarToRow (35) + : +- CometFilter (34) + : +- CometScan parquet spark_catalog.default.item (33) + :- * HashAggregate (72) + : +- * ColumnarToRow (71) + : +- CometColumnarExchange (70) + : +- RowToColumnar (69) + : +- * HashAggregate (68) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (61) + : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : :- * Project (55) + : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : :- * Project (52) + : : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : : :- * Project (49) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : : : :- * ColumnarToRow (46) + : : : : : : : +- CometFilter (45) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (44) + : : : : : : +- ReusedExchange (47) + : : : : : +- ReusedExchange (50) + : : : : +- ReusedExchange (53) + : : : +- BroadcastExchange (59) + : : : +- * ColumnarToRow (58) + : : : +- CometFilter (57) + : : : +- CometScan parquet spark_catalog.default.customer_address (56) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + :- * HashAggregate (102) + : +- * ColumnarToRow (101) + : +- CometColumnarExchange (100) + : +- RowToColumnar (99) + : +- * HashAggregate (98) + : +- * Project (97) + : +- * BroadcastHashJoin Inner BuildRight (96) + : :- * Project (94) + : : +- * BroadcastHashJoin Inner BuildRight (93) + : : :- * Project (91) + : : : +- * BroadcastHashJoin Inner BuildRight (90) + : : : :- * Project (84) + : : : : +- * BroadcastHashJoin Inner BuildRight (83) + : : : : :- * Project (81) + : : : : : +- * BroadcastHashJoin Inner BuildRight (80) + : : : : : :- * Project (78) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) + : : : : : : :- * ColumnarToRow (75) + : : : : : : : +- CometFilter (74) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) + : : : : : : +- ReusedExchange (76) + : : : : : +- ReusedExchange (79) + : : : : +- ReusedExchange (82) + : : : +- BroadcastExchange (89) + : : : +- * ColumnarToRow (88) + : : : +- CometProject (87) + : : : +- CometFilter (86) + : : : +- CometScan parquet spark_catalog.default.customer_address (85) + : : +- ReusedExchange (92) + : +- ReusedExchange (95) + :- * HashAggregate (132) + : +- * ColumnarToRow (131) + : +- CometColumnarExchange (130) + : +- RowToColumnar (129) + : +- * HashAggregate (128) + : +- * Project (127) + : +- * BroadcastHashJoin Inner BuildRight (126) + : :- * Project (124) + : : +- * BroadcastHashJoin Inner BuildRight (123) + : : :- * Project (121) + : : : +- * BroadcastHashJoin Inner BuildRight (120) + : : : :- * Project (114) + : : : : +- * BroadcastHashJoin Inner BuildRight (113) + : : : : :- * Project (111) + : : : : : +- * BroadcastHashJoin Inner BuildRight (110) + : : : : : :- * Project (108) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (107) + : : : : : : :- * ColumnarToRow (105) + : : : : : : : +- CometFilter (104) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (103) + : : : : : : +- ReusedExchange (106) + : : : : : +- ReusedExchange (109) + : : : : +- ReusedExchange (112) + : : : +- BroadcastExchange (119) + : : : +- * ColumnarToRow (118) + : : : +- CometProject (117) + : : : +- CometFilter (116) + : : : +- CometScan parquet spark_catalog.default.customer_address (115) + : : +- ReusedExchange (122) + : +- ReusedExchange (125) + +- * HashAggregate (161) + +- * ColumnarToRow (160) + +- CometColumnarExchange (159) + +- RowToColumnar (158) + +- * HashAggregate (157) + +- * Project (156) + +- * BroadcastHashJoin Inner BuildRight (155) + :- * Project (150) + : +- * BroadcastHashJoin Inner BuildRight (149) + : :- * Project (147) + : : +- * BroadcastHashJoin Inner BuildRight (146) + : : :- * Project (144) + : : : +- * BroadcastHashJoin Inner BuildRight (143) + : : : :- * Project (141) + : : : : +- * BroadcastHashJoin Inner BuildRight (140) + : : : : :- * Project (138) + : : : : : +- * BroadcastHashJoin Inner BuildRight (137) + : : : : : :- * ColumnarToRow (135) + : : : : : : +- CometFilter (134) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (133) + : : : : : +- ReusedExchange (136) + : : : : +- ReusedExchange (139) + : : : +- ReusedExchange (142) + : : +- ReusedExchange (145) + : +- ReusedExchange (148) + +- BroadcastExchange (154) + +- * ColumnarToRow (153) + +- CometFilter (152) + +- CometScan parquet spark_catalog.default.item (151) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -295,7 +299,7 @@ Join condition: None Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -(30) ReusedExchange [Reuses operator id: 164] +(30) ReusedExchange [Reuses operator id: 168] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 7] @@ -350,62 +354,67 @@ Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, cou Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, ca_county#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(42) CometHashAggregate +(42) ColumnarToRow [codegen id : 8] +Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] + +(43) HashAggregate [codegen id : 8] Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Keys [4]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] +Aggregate Attributes [7]: [avg(agg1#28)#63, avg(agg2#29)#64, avg(agg3#30)#65, avg(agg4#31)#66, avg(agg5#32)#67, avg(agg6#33)#68, avg(agg7#34)#69] +Results [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, avg(agg1#28)#63 AS agg1#70, avg(agg2#29)#64 AS agg2#71, avg(agg3#30)#65 AS agg3#72, avg(agg4#31)#66 AS agg4#73, avg(agg5#32)#67 AS agg5#74, avg(agg6#33)#68 AS agg6#75, avg(agg7#34)#69 AS agg7#76] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#77)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(44) CometFilter +(45) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(45) ColumnarToRow [codegen id : 14] +(46) ColumnarToRow [codegen id : 15] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(46) ReusedExchange [Reuses operator id: 8] +(47) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(47) BroadcastHashJoin [codegen id : 14] +(48) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(48) Project [codegen id : 14] +(49) Project [codegen id : 15] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(49) ReusedExchange [Reuses operator id: 15] +(50) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(50) BroadcastHashJoin [codegen id : 14] +(51) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(51) Project [codegen id : 14] +(52) Project [codegen id : 15] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(52) ReusedExchange [Reuses operator id: 21] +(53) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(53) BroadcastHashJoin [codegen id : 14] +(54) BroadcastHashJoin [codegen id : 15] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(54) Project [codegen id : 14] +(55) Project [codegen id : 15] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -416,123 +425,128 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(56) CometFilter +(57) CometFilter Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(57) ColumnarToRow [codegen id : 11] +(58) ColumnarToRow [codegen id : 12] Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] -(58) BroadcastExchange +(59) BroadcastExchange Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(59) BroadcastHashJoin [codegen id : 14] +(60) BroadcastHashJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(60) Project [codegen id : 14] +(61) Project [codegen id : 15] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_state#23, ca_country#24] -(61) ReusedExchange [Reuses operator id: 164] +(62) ReusedExchange [Reuses operator id: 168] Output [1]: [d_date_sk#25] -(62) BroadcastHashJoin [codegen id : 14] +(63) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(63) Project [codegen id : 14] +(64) Project [codegen id : 15] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24, d_date_sk#25] -(64) ReusedExchange [Reuses operator id: 36] +(65) ReusedExchange [Reuses operator id: 36] Output [2]: [i_item_sk#26, i_item_id#27] -(65) BroadcastHashJoin [codegen id : 14] +(66) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(66) Project [codegen id : 14] +(67) Project [codegen id : 15] Output [10]: [i_item_id#27, ca_country#24, ca_state#23, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24, i_item_sk#26, i_item_id#27] -(67) HashAggregate [codegen id : 14] +(68) HashAggregate [codegen id : 15] Input [10]: [i_item_id#27, ca_country#24, ca_state#23, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [3]: [i_item_id#27, ca_country#24, ca_state#23] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71, sum#72, count#73, sum#74, count#75, sum#76, count#77] -Results [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] +Aggregate Attributes [14]: [sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] +Results [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] -(68) RowToColumnar -Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] +(69) RowToColumnar +Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] -(69) CometColumnarExchange -Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] +(70) CometColumnarExchange +Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(70) CometHashAggregate -Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] +(71) ColumnarToRow [codegen id : 16] +Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] + +(72) HashAggregate [codegen id : 16] +Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] Keys [3]: [i_item_id#27, ca_country#24, ca_state#23] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] +Aggregate Attributes [7]: [avg(agg1#28)#106, avg(agg2#29)#107, avg(agg3#30)#108, avg(agg4#31)#109, avg(agg5#32)#110, avg(agg6#33)#111, avg(agg7#34)#112] +Results [11]: [i_item_id#27, ca_country#24, ca_state#23, null AS county#113, avg(agg1#28)#106 AS agg1#114, avg(agg2#29)#107 AS agg2#115, avg(agg3#30)#108 AS agg3#116, avg(agg4#31)#109 AS agg4#117, avg(agg5#32)#110 AS agg5#118, avg(agg6#33)#111 AS agg6#119, avg(agg7#34)#112 AS agg7#120] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#121)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(72) CometFilter +(74) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(73) ColumnarToRow [codegen id : 21] +(75) ColumnarToRow [codegen id : 23] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(74) ReusedExchange [Reuses operator id: 8] +(76) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(75) BroadcastHashJoin [codegen id : 21] +(77) BroadcastHashJoin [codegen id : 23] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(76) Project [codegen id : 21] +(78) Project [codegen id : 23] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(77) ReusedExchange [Reuses operator id: 15] +(79) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(78) BroadcastHashJoin [codegen id : 21] +(80) BroadcastHashJoin [codegen id : 23] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(79) Project [codegen id : 21] +(81) Project [codegen id : 23] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(80) ReusedExchange [Reuses operator id: 21] +(82) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(81) BroadcastHashJoin [codegen id : 21] +(83) BroadcastHashJoin [codegen id : 23] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(82) Project [codegen id : 21] +(84) Project [codegen id : 23] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -543,127 +557,132 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(84) CometFilter +(86) CometFilter Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(85) CometProject +(87) CometProject Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_country#24], [ca_address_sk#21, ca_country#24] -(86) ColumnarToRow [codegen id : 18] +(88) ColumnarToRow [codegen id : 20] Input [2]: [ca_address_sk#21, ca_country#24] -(87) BroadcastExchange +(89) BroadcastExchange Input [2]: [ca_address_sk#21, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(88) BroadcastHashJoin [codegen id : 21] +(90) BroadcastHashJoin [codegen id : 23] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(89) Project [codegen id : 21] +(91) Project [codegen id : 23] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_country#24] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_country#24] -(90) ReusedExchange [Reuses operator id: 164] +(92) ReusedExchange [Reuses operator id: 168] Output [1]: [d_date_sk#25] -(91) BroadcastHashJoin [codegen id : 21] +(93) BroadcastHashJoin [codegen id : 23] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(92) Project [codegen id : 21] +(94) Project [codegen id : 23] Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_country#24, d_date_sk#25] -(93) ReusedExchange [Reuses operator id: 36] +(95) ReusedExchange [Reuses operator id: 36] Output [2]: [i_item_sk#26, i_item_id#27] -(94) BroadcastHashJoin [codegen id : 21] +(96) BroadcastHashJoin [codegen id : 23] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(95) Project [codegen id : 21] +(97) Project [codegen id : 23] Output [9]: [i_item_id#27, ca_country#24, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24, i_item_sk#26, i_item_id#27] -(96) HashAggregate [codegen id : 21] +(98) HashAggregate [codegen id : 23] Input [9]: [i_item_id#27, ca_country#24, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [2]: [i_item_id#27, ca_country#24] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104, sum#105, count#106] -Results [16]: [i_item_id#27, ca_country#24, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +Aggregate Attributes [14]: [sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Results [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] -(97) RowToColumnar -Input [16]: [i_item_id#27, ca_country#24, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +(99) RowToColumnar +Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] -(98) CometColumnarExchange -Input [16]: [i_item_id#27, ca_country#24, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +(100) CometColumnarExchange +Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] Arguments: hashpartitioning(i_item_id#27, ca_country#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(99) CometHashAggregate -Input [16]: [i_item_id#27, ca_country#24, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +(101) ColumnarToRow [codegen id : 24] +Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] + +(102) HashAggregate [codegen id : 24] +Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] Keys [2]: [i_item_id#27, ca_country#24] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] +Aggregate Attributes [7]: [avg(agg1#28)#150, avg(agg2#29)#151, avg(agg3#30)#152, avg(agg4#31)#153, avg(agg5#32)#154, avg(agg6#33)#155, avg(agg7#34)#156] +Results [11]: [i_item_id#27, ca_country#24, null AS ca_state#157, null AS county#158, avg(agg1#28)#150 AS agg1#159, avg(agg2#29)#151 AS agg2#160, avg(agg3#30)#152 AS agg3#161, avg(agg4#31)#153 AS agg4#162, avg(agg5#32)#154 AS agg5#163, avg(agg6#33)#155 AS agg6#164, avg(agg7#34)#156 AS agg7#165] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#121)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#166)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(101) CometFilter +(104) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(102) ColumnarToRow [codegen id : 28] +(105) ColumnarToRow [codegen id : 31] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(103) ReusedExchange [Reuses operator id: 8] +(106) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(104) BroadcastHashJoin [codegen id : 28] +(107) BroadcastHashJoin [codegen id : 31] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(105) Project [codegen id : 28] +(108) Project [codegen id : 31] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(106) ReusedExchange [Reuses operator id: 15] +(109) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(107) BroadcastHashJoin [codegen id : 28] +(110) BroadcastHashJoin [codegen id : 31] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(108) Project [codegen id : 28] +(111) Project [codegen id : 31] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(109) ReusedExchange [Reuses operator id: 21] +(112) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(110) BroadcastHashJoin [codegen id : 28] +(113) BroadcastHashJoin [codegen id : 31] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(111) Project [codegen id : 28] +(114) Project [codegen id : 31] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -674,153 +693,158 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(113) CometFilter +(116) CometFilter Input [2]: [ca_address_sk#21, ca_state#23] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(114) CometProject +(117) CometProject Input [2]: [ca_address_sk#21, ca_state#23] Arguments: [ca_address_sk#21], [ca_address_sk#21] -(115) ColumnarToRow [codegen id : 25] +(118) ColumnarToRow [codegen id : 28] Input [1]: [ca_address_sk#21] -(116) BroadcastExchange +(119) BroadcastExchange Input [1]: [ca_address_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(117) BroadcastHashJoin [codegen id : 28] +(120) BroadcastHashJoin [codegen id : 31] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(118) Project [codegen id : 28] +(121) Project [codegen id : 31] Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21] -(119) ReusedExchange [Reuses operator id: 164] +(122) ReusedExchange [Reuses operator id: 168] Output [1]: [d_date_sk#25] -(120) BroadcastHashJoin [codegen id : 28] +(123) BroadcastHashJoin [codegen id : 31] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(121) Project [codegen id : 28] +(124) Project [codegen id : 31] Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, d_date_sk#25] -(122) ReusedExchange [Reuses operator id: 36] +(125) ReusedExchange [Reuses operator id: 36] Output [2]: [i_item_sk#26, i_item_id#27] -(123) BroadcastHashJoin [codegen id : 28] +(126) BroadcastHashJoin [codegen id : 31] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(124) Project [codegen id : 28] +(127) Project [codegen id : 31] Output [8]: [i_item_id#27, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_sk#26, i_item_id#27] -(125) HashAggregate [codegen id : 28] +(128) HashAggregate [codegen id : 31] Input [8]: [i_item_id#27, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [1]: [i_item_id#27] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] -Results [15]: [i_item_id#27, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] +Aggregate Attributes [14]: [sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180] +Results [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] -(126) RowToColumnar -Input [15]: [i_item_id#27, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] +(129) RowToColumnar +Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] -(127) CometColumnarExchange -Input [15]: [i_item_id#27, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] +(130) CometColumnarExchange +Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(128) CometHashAggregate -Input [15]: [i_item_id#27, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] +(131) ColumnarToRow [codegen id : 32] +Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] + +(132) HashAggregate [codegen id : 32] +Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] Keys [1]: [i_item_id#27] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] +Aggregate Attributes [7]: [avg(agg1#28)#195, avg(agg2#29)#196, avg(agg3#30)#197, avg(agg4#31)#198, avg(agg5#32)#199, avg(agg6#33)#200, avg(agg7#34)#201] +Results [11]: [i_item_id#27, null AS ca_country#202, null AS ca_state#203, null AS county#204, avg(agg1#28)#195 AS agg1#205, avg(agg2#29)#196 AS agg2#206, avg(agg3#30)#197 AS agg3#207, avg(agg4#31)#198 AS agg4#208, avg(agg5#32)#199 AS agg5#209, avg(agg6#33)#200 AS agg6#210, avg(agg7#34)#201 AS agg7#211] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#150)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#212)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(130) CometFilter +(134) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(131) ColumnarToRow [codegen id : 35] +(135) ColumnarToRow [codegen id : 39] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(132) ReusedExchange [Reuses operator id: 8] +(136) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(133) BroadcastHashJoin [codegen id : 35] +(137) BroadcastHashJoin [codegen id : 39] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(134) Project [codegen id : 35] +(138) Project [codegen id : 39] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(135) ReusedExchange [Reuses operator id: 15] +(139) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(136) BroadcastHashJoin [codegen id : 35] +(140) BroadcastHashJoin [codegen id : 39] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(137) Project [codegen id : 35] +(141) Project [codegen id : 39] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(138) ReusedExchange [Reuses operator id: 21] +(142) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(139) BroadcastHashJoin [codegen id : 35] +(143) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(140) Project [codegen id : 35] +(144) Project [codegen id : 39] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -(141) ReusedExchange [Reuses operator id: 116] +(145) ReusedExchange [Reuses operator id: 119] Output [1]: [ca_address_sk#21] -(142) BroadcastHashJoin [codegen id : 35] +(146) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(143) Project [codegen id : 35] +(147) Project [codegen id : 39] Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21] -(144) ReusedExchange [Reuses operator id: 164] +(148) ReusedExchange [Reuses operator id: 168] Output [1]: [d_date_sk#25] -(145) BroadcastHashJoin [codegen id : 35] +(149) BroadcastHashJoin [codegen id : 39] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(146) Project [codegen id : 35] +(150) Project [codegen id : 39] Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, d_date_sk#25] @@ -831,98 +855,95 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(148) CometFilter +(152) CometFilter Input [1]: [i_item_sk#26] Condition : isnotnull(i_item_sk#26) -(149) ColumnarToRow [codegen id : 34] +(153) ColumnarToRow [codegen id : 38] Input [1]: [i_item_sk#26] -(150) BroadcastExchange +(154) BroadcastExchange Input [1]: [i_item_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(151) BroadcastHashJoin [codegen id : 35] +(155) BroadcastHashJoin [codegen id : 39] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(152) Project [codegen id : 35] +(156) Project [codegen id : 39] Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_sk#26] -(153) HashAggregate [codegen id : 35] +(157) HashAggregate [codegen id : 39] Input [7]: [agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys: [] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164] -Results [14]: [sum#165, count#166, sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178] +Aggregate Attributes [14]: [sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224, sum#225, count#226] +Results [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] -(154) RowToColumnar -Input [14]: [sum#165, count#166, sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178] +(158) RowToColumnar +Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] -(155) CometColumnarExchange -Input [14]: [sum#165, count#166, sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178] +(159) CometColumnarExchange +Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(156) CometHashAggregate -Input [14]: [sum#165, count#166, sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178] +(160) ColumnarToRow [codegen id : 40] +Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] + +(161) HashAggregate [codegen id : 40] +Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] Keys: [] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] +Aggregate Attributes [7]: [avg(agg1#28)#241, avg(agg2#29)#242, avg(agg3#30)#243, avg(agg4#31)#244, avg(agg5#32)#245, avg(agg6#33)#246, avg(agg7#34)#247] +Results [11]: [null AS i_item_id#248, null AS ca_country#249, null AS ca_state#250, null AS county#251, avg(agg1#28)#241 AS agg1#252, avg(agg2#29)#242 AS agg2#253, avg(agg3#30)#243 AS agg3#254, avg(agg4#31)#244 AS agg4#255, avg(agg5#32)#245 AS agg5#256, avg(agg6#33)#246 AS agg6#257, avg(agg7#34)#247 AS agg7#258] -(157) CometUnion -Child 0 Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#179, agg2#180, agg3#181, agg4#182, agg5#183, agg6#184, agg7#185] -Child 1 Input [11]: [i_item_id#27, ca_country#24, ca_state#23, county#186, agg1#187, agg2#188, agg3#189, agg4#190, agg5#191, agg6#192, agg7#193] -Child 2 Input [11]: [i_item_id#27, ca_country#24, ca_state#194, county#195, agg1#196, agg2#197, agg3#198, agg4#199, agg5#200, agg6#201, agg7#202] -Child 3 Input [11]: [i_item_id#27, ca_country#203, ca_state#204, county#205, agg1#206, agg2#207, agg3#208, agg4#209, agg5#210, agg6#211, agg7#212] -Child 4 Input [11]: [i_item_id#213, ca_country#214, ca_state#215, county#216, agg1#217, agg2#218, agg3#219, agg4#220, agg5#221, agg6#222, agg7#223] - -(158) CometTakeOrderedAndProject -Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#179, agg2#180, agg3#181, agg4#182, agg5#183, agg6#184, agg7#185] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#24 ASC NULLS FIRST,ca_state#23 ASC NULLS FIRST,ca_county#22 ASC NULLS FIRST,i_item_id#27 ASC NULLS FIRST], output=[i_item_id#27,ca_country#24,ca_state#23,ca_county#22,agg1#179,agg2#180,agg3#181,agg4#182,agg5#183,agg6#184,agg7#185]), 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#27 ASC NULLS FIRST], [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#179, agg2#180, agg3#181, agg4#182, agg5#183, agg6#184, agg7#185] +(162) Union -(159) ColumnarToRow [codegen id : 36] -Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#179, agg2#180, agg3#181, agg4#182, agg5#183, agg6#184, agg7#185] +(163) TakeOrderedAndProject +Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] +Arguments: 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#27 ASC NULLS FIRST], [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (164) -+- * ColumnarToRow (163) - +- CometProject (162) - +- CometFilter (161) - +- CometScan parquet spark_catalog.default.date_dim (160) +BroadcastExchange (168) ++- * ColumnarToRow (167) + +- CometProject (166) + +- CometFilter (165) + +- CometScan parquet spark_catalog.default.date_dim (164) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#224] +Output [2]: [d_date_sk#25, d_year#259] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(161) CometFilter -Input [2]: [d_date_sk#25, d_year#224] -Condition : ((isnotnull(d_year#224) AND (d_year#224 = 2001)) AND isnotnull(d_date_sk#25)) +(165) CometFilter +Input [2]: [d_date_sk#25, d_year#259] +Condition : ((isnotnull(d_year#259) AND (d_year#259 = 2001)) AND isnotnull(d_date_sk#25)) -(162) CometProject -Input [2]: [d_date_sk#25, d_year#224] +(166) CometProject +Input [2]: [d_date_sk#25, d_year#259] Arguments: [d_date_sk#25], [d_date_sk#25] -(163) ColumnarToRow [codegen id : 1] +(167) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(164) BroadcastExchange +(168) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:2 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 71 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 100 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 133 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index e0eebcd657..cc38540a22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (36) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometUnion - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] +TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + Union + WholeStageCodegen (8) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 RowToColumnar WholeStageCodegen (7) @@ -71,10 +71,13 @@ WholeStageCodegen (36) InputAdapter CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + WholeStageCodegen (16) + HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id,ca_country,ca_state] #8 RowToColumnar - WholeStageCodegen (14) + WholeStageCodegen (15) HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -101,7 +104,7 @@ WholeStageCodegen (36) ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #9 - WholeStageCodegen (11) + WholeStageCodegen (12) ColumnarToRow InputAdapter CometFilter [ca_state,ca_address_sk] @@ -110,10 +113,13 @@ WholeStageCodegen (36) ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + WholeStageCodegen (24) + HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id,ca_country] #10 RowToColumnar - WholeStageCodegen (21) + WholeStageCodegen (23) HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -140,7 +146,7 @@ WholeStageCodegen (36) ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #11 - WholeStageCodegen (18) + WholeStageCodegen (20) ColumnarToRow InputAdapter CometProject [ca_address_sk,ca_country] @@ -150,10 +156,13 @@ WholeStageCodegen (36) ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + WholeStageCodegen (32) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #12 RowToColumnar - WholeStageCodegen (28) + WholeStageCodegen (31) HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -180,7 +189,7 @@ WholeStageCodegen (36) ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #13 - WholeStageCodegen (25) + WholeStageCodegen (28) ColumnarToRow InputAdapter CometProject [ca_address_sk] @@ -190,10 +199,13 @@ WholeStageCodegen (36) ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + WholeStageCodegen (40) + HashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange #14 RowToColumnar - WholeStageCodegen (35) + WholeStageCodegen (39) HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -224,7 +236,7 @@ WholeStageCodegen (36) ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #15 - WholeStageCodegen (34) + WholeStageCodegen (38) ColumnarToRow InputAdapter CometFilter [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index f62cbd1f86..9405af5bd9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -1,26 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * Project (21) - +- Window (20) - +- * ColumnarToRow (19) - +- CometSort (18) - +- CometColumnarExchange (17) - +- CometHashAggregate (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * ColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- RowToColumnar (18) + +- * HashAggregate (17) + +- * ColumnarToRow (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -66,7 +68,7 @@ Join condition: None Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 27] +(10) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -93,63 +95,71 @@ Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] +Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16] + +(18) RowToColumnar +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -(17) CometColumnarExchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +(19) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) CometSort -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] -Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] +(20) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 4] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +(21) ColumnarToRow [codegen id : 5] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -(20) Window -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] -Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] +(22) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(21) Project [codegen id : 5] -Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] -Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] +(23) Project [codegen id : 6] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] -(22) TakeOrderedAndProject -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +(24) TakeOrderedAndProject +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#18] +Output [2]: [d_date_sk#11, d_date#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [d_date_sk#11, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(26) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(25) CometProject -Input [2]: [d_date_sk#11, d_date#18] +(27) CometProject +Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index 4cf648ccf6..0cc4015f20 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -1,40 +1,44 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (5) + WholeStageCodegen (6) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (4) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometSort [i_class] CometColumnarExchange [i_class] #1 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt index 9328d2b49d..0bca5ae8cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometHashAggregate (22) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- * ColumnarToRow (22) +- CometColumnarExchange (21) +- RowToColumnar (20) +- * HashAggregate (19) @@ -120,17 +120,19 @@ Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grou Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] Arguments: hashpartitioning(i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometHashAggregate +(22) ColumnarToRow [codegen id : 5] +Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] + +(23) HashAggregate [codegen id : 5] Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] Keys [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] Functions [1]: [avg(inv_quantity_on_hand#2)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#20] +Results [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, avg(inv_quantity_on_hand#2)#20 AS qoh#21] -(23) CometTakeOrderedAndProject -Input [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#20] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#20 ASC NULLS FIRST,i_product_name#11 ASC NULLS FIRST,i_brand#12 ASC NULLS FIRST,i_class#13 ASC NULLS FIRST,i_category#14 ASC NULLS FIRST], output=[i_product_name#11,i_brand#12,i_class#13,i_category#14,qoh#20]), 100, [qoh#20 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#12 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_category#14 ASC NULLS FIRST], [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#20] - -(24) ColumnarToRow [codegen id : 5] -Input [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#20] +(24) TakeOrderedAndProject +Input [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] +Arguments: 100, [qoh#21 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#12 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_category#14 ASC NULLS FIRST], [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] ===== Subqueries ===== @@ -143,18 +145,18 @@ BroadcastExchange (29) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#21] +Output [2]: [d_date_sk#5, d_month_seq#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct (26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#21] -Condition : (((isnotnull(d_month_seq#21) AND (d_month_seq#21 >= 1200)) AND (d_month_seq#21 <= 1211)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#5, d_month_seq#22] +Condition : (((isnotnull(d_month_seq#22) AND (d_month_seq#22 >= 1200)) AND (d_month_seq#22 <= 1211)) AND isnotnull(d_date_sk#5)) (27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#21] +Input [2]: [d_date_sk#5, d_month_seq#22] Arguments: [d_date_sk#5], [d_date_sk#5] (28) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 797b870040..2640e2dc58 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] + WholeStageCodegen (5) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 RowToColumnar WholeStageCodegen (4) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index 3911ebef4c..98d08fb2f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -1,51 +1,63 @@ == Physical Plan == -* ColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometUnion (45) - :- CometHashAggregate (24) - : +- CometHashAggregate (23) - : +- CometHashAggregate (22) - : +- CometColumnarExchange (21) - : +- RowToColumnar (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.warehouse (13) - :- CometHashAggregate (29) - : +- CometColumnarExchange (28) - : +- CometHashAggregate (27) - : +- CometHashAggregate (26) - : +- ReusedExchange (25) - :- CometHashAggregate (34) - : +- CometColumnarExchange (33) - : +- CometHashAggregate (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - :- CometHashAggregate (39) - : +- CometColumnarExchange (38) - : +- CometHashAggregate (37) - : +- CometHashAggregate (36) - : +- ReusedExchange (35) - +- CometHashAggregate (44) - +- CometColumnarExchange (43) - +- CometHashAggregate (42) - +- CometHashAggregate (41) - +- ReusedExchange (40) +TakeOrderedAndProject (59) ++- Union (58) + :- * HashAggregate (25) + : +- * HashAggregate (24) + : +- * HashAggregate (23) + : +- * ColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- RowToColumnar (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.item (7) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.warehouse (13) + :- * HashAggregate (33) + : +- * ColumnarToRow (32) + : +- CometColumnarExchange (31) + : +- RowToColumnar (30) + : +- * HashAggregate (29) + : +- * HashAggregate (28) + : +- * ColumnarToRow (27) + : +- ReusedExchange (26) + :- * HashAggregate (41) + : +- * ColumnarToRow (40) + : +- CometColumnarExchange (39) + : +- RowToColumnar (38) + : +- * HashAggregate (37) + : +- * HashAggregate (36) + : +- * ColumnarToRow (35) + : +- ReusedExchange (34) + :- * HashAggregate (49) + : +- * ColumnarToRow (48) + : +- CometColumnarExchange (47) + : +- RowToColumnar (46) + : +- * HashAggregate (45) + : +- * HashAggregate (44) + : +- * ColumnarToRow (43) + : +- ReusedExchange (42) + +- * HashAggregate (57) + +- * ColumnarToRow (56) + +- CometColumnarExchange (55) + +- RowToColumnar (54) + +- * HashAggregate (53) + +- * HashAggregate (52) + +- * ColumnarToRow (51) + +- ReusedExchange (50) (unknown) Scan parquet spark_catalog.default.inventory @@ -63,7 +75,7 @@ Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (3) ColumnarToRow [codegen id : 4] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(4) ReusedExchange [Reuses operator id: 52] +(4) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 4] @@ -146,152 +158,213 @@ Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, coun Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometHashAggregate +(22) ColumnarToRow [codegen id : 5] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] + +(23) HashAggregate [codegen id : 5] Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] +Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(23) CometHashAggregate -Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#17] +(24) HashAggregate [codegen id : 5] +Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#18] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] -Functions [1]: [partial_avg(qoh#17)] +Functions [1]: [partial_avg(qoh#18)] +Aggregate Attributes [2]: [sum#19, count#20] +Results [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] -(24) CometHashAggregate -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#18, count#19] +(25) HashAggregate [codegen id : 5] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] -Functions [1]: [avg(qoh#17)] +Functions [1]: [avg(qoh#18)] +Aggregate Attributes [1]: [avg(qoh#18)#23] +Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(qoh#18)#23 AS qoh#24] -(25) ReusedExchange [Reuses operator id: 21] +(26) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(26) CometHashAggregate +(27) ColumnarToRow [codegen id : 10] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] + +(28) HashAggregate [codegen id : 10] Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] +Results [4]: [i_product_name#11, i_brand#8, i_class#9, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(27) CometHashAggregate -Input [4]: [i_product_name#11, i_brand#8, i_class#9, qoh#17] +(29) HashAggregate [codegen id : 10] +Input [4]: [i_product_name#11, i_brand#8, i_class#9, qoh#18] Keys [3]: [i_product_name#11, i_brand#8, i_class#9] -Functions [1]: [partial_avg(qoh#17)] +Functions [1]: [partial_avg(qoh#18)] +Aggregate Attributes [2]: [sum#25, count#26] +Results [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] + +(30) RowToColumnar +Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] -(28) CometColumnarExchange -Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#20, count#21] +(31) CometColumnarExchange +Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometHashAggregate -Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#20, count#21] +(32) ColumnarToRow [codegen id : 11] +Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] + +(33) HashAggregate [codegen id : 11] +Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] Keys [3]: [i_product_name#11, i_brand#8, i_class#9] -Functions [1]: [avg(qoh#17)] +Functions [1]: [avg(qoh#18)] +Aggregate Attributes [1]: [avg(qoh#18)#29] +Results [5]: [i_product_name#11, i_brand#8, i_class#9, null AS i_category#30, avg(qoh#18)#29 AS qoh#31] -(30) ReusedExchange [Reuses operator id: 21] +(34) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(31) CometHashAggregate +(35) ColumnarToRow [codegen id : 16] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] + +(36) HashAggregate [codegen id : 16] Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] +Results [3]: [i_product_name#11, i_brand#8, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(32) CometHashAggregate -Input [3]: [i_product_name#11, i_brand#8, qoh#17] +(37) HashAggregate [codegen id : 16] +Input [3]: [i_product_name#11, i_brand#8, qoh#18] Keys [2]: [i_product_name#11, i_brand#8] -Functions [1]: [partial_avg(qoh#17)] +Functions [1]: [partial_avg(qoh#18)] +Aggregate Attributes [2]: [sum#32, count#33] +Results [4]: [i_product_name#11, i_brand#8, sum#34, count#35] -(33) CometColumnarExchange -Input [4]: [i_product_name#11, i_brand#8, sum#22, count#23] +(38) RowToColumnar +Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] + +(39) CometColumnarExchange +Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] Arguments: hashpartitioning(i_product_name#11, i_brand#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(34) CometHashAggregate -Input [4]: [i_product_name#11, i_brand#8, sum#22, count#23] +(40) ColumnarToRow [codegen id : 17] +Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] + +(41) HashAggregate [codegen id : 17] +Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] Keys [2]: [i_product_name#11, i_brand#8] -Functions [1]: [avg(qoh#17)] +Functions [1]: [avg(qoh#18)] +Aggregate Attributes [1]: [avg(qoh#18)#36] +Results [5]: [i_product_name#11, i_brand#8, null AS i_class#37, null AS i_category#38, avg(qoh#18)#36 AS qoh#39] -(35) ReusedExchange [Reuses operator id: 21] +(42) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(36) CometHashAggregate +(43) ColumnarToRow [codegen id : 22] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] + +(44) HashAggregate [codegen id : 22] Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] +Results [2]: [i_product_name#11, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(37) CometHashAggregate -Input [2]: [i_product_name#11, qoh#17] +(45) HashAggregate [codegen id : 22] +Input [2]: [i_product_name#11, qoh#18] Keys [1]: [i_product_name#11] -Functions [1]: [partial_avg(qoh#17)] +Functions [1]: [partial_avg(qoh#18)] +Aggregate Attributes [2]: [sum#40, count#41] +Results [3]: [i_product_name#11, sum#42, count#43] + +(46) RowToColumnar +Input [3]: [i_product_name#11, sum#42, count#43] -(38) CometColumnarExchange -Input [3]: [i_product_name#11, sum#24, count#25] +(47) CometColumnarExchange +Input [3]: [i_product_name#11, sum#42, count#43] Arguments: hashpartitioning(i_product_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(39) CometHashAggregate -Input [3]: [i_product_name#11, sum#24, count#25] +(48) ColumnarToRow [codegen id : 23] +Input [3]: [i_product_name#11, sum#42, count#43] + +(49) HashAggregate [codegen id : 23] +Input [3]: [i_product_name#11, sum#42, count#43] Keys [1]: [i_product_name#11] -Functions [1]: [avg(qoh#17)] +Functions [1]: [avg(qoh#18)] +Aggregate Attributes [1]: [avg(qoh#18)#44] +Results [5]: [i_product_name#11, null AS i_brand#45, null AS i_class#46, null AS i_category#47, avg(qoh#18)#44 AS qoh#48] -(40) ReusedExchange [Reuses operator id: 21] +(50) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(41) CometHashAggregate +(51) ColumnarToRow [codegen id : 28] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] + +(52) HashAggregate [codegen id : 28] Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] +Results [1]: [avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(42) CometHashAggregate -Input [1]: [qoh#17] +(53) HashAggregate [codegen id : 28] +Input [1]: [qoh#18] Keys: [] -Functions [1]: [partial_avg(qoh#17)] +Functions [1]: [partial_avg(qoh#18)] +Aggregate Attributes [2]: [sum#49, count#50] +Results [2]: [sum#51, count#52] + +(54) RowToColumnar +Input [2]: [sum#51, count#52] -(43) CometColumnarExchange -Input [2]: [sum#26, count#27] +(55) CometColumnarExchange +Input [2]: [sum#51, count#52] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(44) CometHashAggregate -Input [2]: [sum#26, count#27] -Keys: [] -Functions [1]: [avg(qoh#17)] +(56) ColumnarToRow [codegen id : 29] +Input [2]: [sum#51, count#52] -(45) CometUnion -Child 0 Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#28] -Child 1 Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#29, qoh#30] -Child 2 Input [5]: [i_product_name#11, i_brand#8, i_class#31, i_category#32, qoh#33] -Child 3 Input [5]: [i_product_name#11, i_brand#34, i_class#35, i_category#36, qoh#37] -Child 4 Input [5]: [i_product_name#38, i_brand#39, i_class#40, i_category#41, qoh#42] +(57) HashAggregate [codegen id : 29] +Input [2]: [sum#51, count#52] +Keys: [] +Functions [1]: [avg(qoh#18)] +Aggregate Attributes [1]: [avg(qoh#18)#53] +Results [5]: [null AS i_product_name#54, null AS i_brand#55, null AS i_class#56, null AS i_category#57, avg(qoh#18)#53 AS qoh#58] -(46) CometTakeOrderedAndProject -Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#28 ASC NULLS FIRST,i_product_name#11 ASC NULLS FIRST,i_brand#8 ASC NULLS FIRST,i_class#9 ASC NULLS FIRST,i_category#10 ASC NULLS FIRST], output=[i_product_name#11,i_brand#8,i_class#9,i_category#10,qoh#28]), 100, [qoh#28 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#28] +(58) Union -(47) ColumnarToRow [codegen id : 21] -Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#28] +(59) TakeOrderedAndProject +Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] +Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (64) ++- * ColumnarToRow (63) + +- CometProject (62) + +- CometFilter (61) + +- CometScan parquet spark_catalog.default.date_dim (60) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#43] +Output [2]: [d_date_sk#6, d_month_seq#59] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#43] -Condition : (((isnotnull(d_month_seq#43) AND (d_month_seq#43 >= 1212)) AND (d_month_seq#43 <= 1223)) AND isnotnull(d_date_sk#6)) +(61) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#59] +Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1212)) AND (d_month_seq#59 <= 1223)) AND isnotnull(d_date_sk#6)) -(50) CometProject -Input [2]: [d_date_sk#6, d_month_seq#43] +(62) CometProject +Input [2]: [d_date_sk#6, d_month_seq#59] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) ColumnarToRow [codegen id : 1] +(63) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(52) BroadcastExchange +(64) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 442e169eb9..a614a9093d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -1,11 +1,11 @@ -WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - CometUnion - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] + Union + WholeStageCodegen (5) + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 RowToColumnar WholeStageCodegen (4) @@ -44,23 +44,51 @@ WholeStageCodegen (21) InputAdapter CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [i_product_name,i_brand,i_class,sum,count] + WholeStageCodegen (11) + HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_product_name,i_brand,i_class] #5 - CometHashAggregate [i_product_name,i_brand,i_class,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [i_product_name,i_brand,sum,count] + RowToColumnar + WholeStageCodegen (10) + HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + ColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (17) + HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_product_name,i_brand] #6 - CometHashAggregate [i_product_name,i_brand,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [i_product_name,sum,count] + RowToColumnar + WholeStageCodegen (16) + HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + ColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (23) + HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_product_name] #7 - CometHashAggregate [i_product_name,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] + RowToColumnar + WholeStageCodegen (22) + HashAggregate [i_product_name,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + ColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (29) + HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange #8 - CometHashAggregate [qoh] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + RowToColumnar + WholeStageCodegen (28) + HashAggregate [qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + ColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index c19e023683..325b853594 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -1,53 +1,57 @@ == Physical Plan == -* ColumnarToRow (49) -+- CometSort (48) - +- CometColumnarExchange (47) - +- CometFilter (46) - +- CometHashAggregate (45) - +- CometColumnarExchange (44) - +- CometHashAggregate (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- RowToColumnar (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometColumnarExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometColumnarExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.item (21) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_address (33) +* ColumnarToRow (53) ++- CometSort (52) + +- CometColumnarExchange (51) + +- RowToColumnar (50) + +- * Filter (49) + +- * HashAggregate (48) + +- * ColumnarToRow (47) + +- CometColumnarExchange (46) + +- RowToColumnar (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- * ColumnarToRow (42) + +- CometColumnarExchange (41) + +- RowToColumnar (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * ColumnarToRow (13) + : : : : +- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometColumnarExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometColumnarExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.store (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.item (21) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_address (33) (unknown) Scan parquet spark_catalog.default.store_sales @@ -238,111 +242,131 @@ Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_st Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(42) CometHashAggregate +(42) ColumnarToRow [codegen id : 6] +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] + +(43) HashAggregate [codegen id : 6] Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] +Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] -(43) CometHashAggregate -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, netpaid#32] +(44) HashAggregate [codegen id : 6] +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, netpaid#33] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] -Functions [1]: [partial_sum(netpaid#32)] +Functions [1]: [partial_sum(netpaid#33)] +Aggregate Attributes [2]: [sum#34, isEmpty#35] +Results [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] + +(45) RowToColumnar +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] -(44) CometColumnarExchange -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#33, isEmpty#34] +(46) CometColumnarExchange +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(45) CometHashAggregate -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#33, isEmpty#34] +(47) ColumnarToRow [codegen id : 7] +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] + +(48) HashAggregate [codegen id : 7] +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] -Functions [1]: [sum(netpaid#32)] +Functions [1]: [sum(netpaid#33)] +Aggregate Attributes [1]: [sum(netpaid#33)#38] +Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, sum(netpaid#33)#38 AS paid#39] + +(49) Filter [codegen id : 7] +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] +Condition : (isnotnull(paid#39) AND (cast(paid#39 as decimal(33,8)) > cast(Subquery scalar-subquery#40, [id=#41] as decimal(33,8)))) -(46) CometFilter -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#35] -Condition : (isnotnull(paid#35) AND (cast(paid#35 as decimal(33,8)) > cast(Subquery scalar-subquery#36, [id=#37] as decimal(33,8)))) +(50) RowToColumnar +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -(47) CometColumnarExchange -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#35] +(51) CometColumnarExchange +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(48) CometSort -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#35] -Arguments: [c_last_name#24, c_first_name#23, s_store_name#11, paid#35], [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] +(52) CometSort +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] +Arguments: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39], [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(49) ColumnarToRow [codegen id : 6] -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#35] +(53) ColumnarToRow [codegen id : 8] +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#36, [id=#37] -* ColumnarToRow (79) -+- CometHashAggregate (78) - +- CometColumnarExchange (77) - +- CometHashAggregate (76) - +- CometHashAggregate (75) - +- CometColumnarExchange (74) - +- RowToColumnar (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * ColumnarToRow (56) - : : : : +- CometProject (55) - : : : : +- CometSortMergeJoin (54) - : : : : :- CometSort (51) - : : : : : +- ReusedExchange (50) - : : : : +- CometSort (53) - : : : : +- ReusedExchange (52) - : : : +- ReusedExchange (57) - : : +- BroadcastExchange (63) - : : +- * ColumnarToRow (62) - : : +- CometFilter (61) - : : +- CometScan parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (69) - - -(50) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +* HashAggregate (85) ++- * ColumnarToRow (84) + +- CometColumnarExchange (83) + +- RowToColumnar (82) + +- * HashAggregate (81) + +- * HashAggregate (80) + +- * ColumnarToRow (79) + +- CometColumnarExchange (78) + +- RowToColumnar (77) + +- * HashAggregate (76) + +- * Project (75) + +- * BroadcastHashJoin Inner BuildRight (74) + :- * Project (72) + : +- * BroadcastHashJoin Inner BuildRight (71) + : :- * Project (69) + : : +- * BroadcastHashJoin Inner BuildRight (68) + : : :- * Project (63) + : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : :- * ColumnarToRow (60) + : : : : +- CometProject (59) + : : : : +- CometSortMergeJoin (58) + : : : : :- CometSort (55) + : : : : : +- ReusedExchange (54) + : : : : +- CometSort (57) + : : : : +- ReusedExchange (56) + : : : +- ReusedExchange (61) + : : +- BroadcastExchange (67) + : : +- * ColumnarToRow (66) + : : +- CometFilter (65) + : : +- CometScan parquet spark_catalog.default.item (64) + : +- ReusedExchange (70) + +- ReusedExchange (73) + + +(54) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(51) CometSort +(55) CometSort Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] -(52) ReusedExchange [Reuses operator id: 9] +(56) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(53) CometSort +(57) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(54) CometSortMergeJoin +(58) CometSortMergeJoin Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] Right output [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner -(55) CometProject +(59) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(56) ColumnarToRow [codegen id : 5] +(60) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(57) ReusedExchange [Reuses operator id: 18] +(61) ReusedExchange [Reuses operator id: 18] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(58) BroadcastHashJoin [codegen id : 5] +(62) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(59) Project [codegen id : 5] +(63) Project [codegen id : 5] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -353,87 +377,99 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(61) CometFilter +(65) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : isnotnull(i_item_sk#15) -(62) ColumnarToRow [codegen id : 2] +(66) ColumnarToRow [codegen id : 2] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(63) BroadcastExchange +(67) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(64) BroadcastHashJoin [codegen id : 5] +(68) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(65) Project [codegen id : 5] +(69) Project [codegen id : 5] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(66) ReusedExchange [Reuses operator id: 30] +(70) ReusedExchange [Reuses operator id: 30] Output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(67) BroadcastHashJoin [codegen id : 5] +(71) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(68) Project [codegen id : 5] +(72) Project [codegen id : 5] Output [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(69) ReusedExchange [Reuses operator id: 36] +(73) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(70) BroadcastHashJoin [codegen id : 5] +(74) BroadcastHashJoin [codegen id : 5] Left keys [3]: [c_current_addr_sk#22, c_birth_country#25, s_zip#14] Right keys [3]: [ca_address_sk#26, upper(ca_country#29), ca_zip#28] Join type: Inner Join condition: None -(71) Project [codegen id : 5] +(75) Project [codegen id : 5] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25, ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(72) HashAggregate [codegen id : 5] +(76) HashAggregate [codegen id : 5] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#38] -Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#39] +Aggregate Attributes [1]: [sum#42] +Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] -(73) RowToColumnar -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#39] +(77) RowToColumnar +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] -(74) CometColumnarExchange -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#39] +(78) CometColumnarExchange +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(75) CometHashAggregate -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#39] +(79) ColumnarToRow [codegen id : 6] +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] + +(80) HashAggregate [codegen id : 6] +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] -(76) CometHashAggregate -Input [1]: [netpaid#32] +(81) HashAggregate [codegen id : 6] +Input [1]: [netpaid#33] Keys: [] -Functions [1]: [partial_avg(netpaid#32)] +Functions [1]: [partial_avg(netpaid#33)] +Aggregate Attributes [2]: [sum#44, count#45] +Results [2]: [sum#46, count#47] + +(82) RowToColumnar +Input [2]: [sum#46, count#47] -(77) CometColumnarExchange -Input [2]: [sum#40, count#41] +(83) CometColumnarExchange +Input [2]: [sum#46, count#47] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(78) CometHashAggregate -Input [2]: [sum#40, count#41] -Keys: [] -Functions [1]: [avg(netpaid#32)] +(84) ColumnarToRow [codegen id : 7] +Input [2]: [sum#46, count#47] -(79) ColumnarToRow [codegen id : 6] -Input [1]: [(0.05 * avg(netpaid))#42] +(85) HashAggregate [codegen id : 7] +Input [2]: [sum#46, count#47] +Keys: [] +Functions [1]: [avg(netpaid#33)] +Aggregate Attributes [1]: [avg(netpaid#33)#48] +Results [1]: [(0.05 * avg(netpaid#33)#48) AS (0.05 * avg(netpaid))#49] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index 05de849be4..08ef232256 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -1,106 +1,118 @@ -WholeStageCodegen (6) +WholeStageCodegen (8) ColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - CometFilter [paid] - Subquery #1 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometHashAggregate [sum,count] - CometColumnarExchange #10 - CometHashAggregate [netpaid] - CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 + RowToColumnar + WholeStageCodegen (7) + Filter [paid] + Subquery #1 + WholeStageCodegen (7) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + ColumnarToRow + InputAdapter + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 + CometSort [sr_ticket_number,sr_item_sk] + ReusedExchange [sr_item_sk,sr_ticket_number] #5 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) - ColumnarToRow + ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #4 + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometSort [sr_ticket_number,sr_item_sk] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #5 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - CometHashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - CometHashAggregate [c_last_name,c_first_name,s_store_name,netpaid] - CometHashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter - CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow + BroadcastExchange #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - CometFilter [i_color,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + BroadcastExchange #9 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_country,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt index d689cb65a9..5567524669 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt @@ -1,81 +1,83 @@ == Physical Plan == -* ColumnarToRow (77) -+- CometTakeOrderedAndProject (76) - +- CometUnion (75) - :- CometHashAggregate (29) - : +- CometColumnarExchange (28) - : +- RowToColumnar (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.item (20) - :- CometHashAggregate (52) - : +- CometColumnarExchange (51) - : +- RowToColumnar (50) - : +- * HashAggregate (49) - : +- * Project (48) - : +- * BroadcastHashJoin Inner BuildRight (47) - : :- * Project (45) - : : +- * BroadcastHashJoin Inner BuildRight (44) - : : :- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * ColumnarToRow (32) - : : : : : +- CometFilter (31) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (33) - : : : +- ReusedExchange (36) - : : +- BroadcastExchange (43) - : : +- * ColumnarToRow (42) - : : +- CometProject (41) - : : +- CometFilter (40) - : : +- CometScan parquet spark_catalog.default.store (39) - : +- ReusedExchange (46) - +- CometHashAggregate (74) - +- CometColumnarExchange (73) - +- RowToColumnar (72) - +- * HashAggregate (71) - +- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (58) - : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : :- * ColumnarToRow (55) - : : : : +- CometFilter (54) - : : : : +- CometScan parquet spark_catalog.default.store_sales (53) - : : : +- ReusedExchange (56) - : : +- ReusedExchange (59) - : +- ReusedExchange (62) - +- BroadcastExchange (68) - +- * ColumnarToRow (67) - +- CometFilter (66) - +- CometScan parquet spark_catalog.default.item (65) +TakeOrderedAndProject (79) ++- Union (78) + :- * HashAggregate (30) + : +- * ColumnarToRow (29) + : +- CometColumnarExchange (28) + : +- RowToColumnar (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (17) + : : +- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.store (14) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + :- * HashAggregate (54) + : +- * ColumnarToRow (53) + : +- CometColumnarExchange (52) + : +- RowToColumnar (51) + : +- * HashAggregate (50) + : +- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (46) + : : +- * BroadcastHashJoin Inner BuildRight (45) + : : :- * Project (39) + : : : +- * BroadcastHashJoin Inner BuildRight (38) + : : : :- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * ColumnarToRow (33) + : : : : : +- CometFilter (32) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (31) + : : : : +- ReusedExchange (34) + : : : +- ReusedExchange (37) + : : +- BroadcastExchange (44) + : : +- * ColumnarToRow (43) + : : +- CometProject (42) + : : +- CometFilter (41) + : : +- CometScan parquet spark_catalog.default.store (40) + : +- ReusedExchange (47) + +- * HashAggregate (77) + +- * ColumnarToRow (76) + +- CometColumnarExchange (75) + +- RowToColumnar (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : :- * ColumnarToRow (57) + : : : : +- CometFilter (56) + : : : : +- CometScan parquet spark_catalog.default.store_sales (55) + : : : +- ReusedExchange (58) + : : +- ReusedExchange (61) + : +- ReusedExchange (64) + +- BroadcastExchange (70) + +- * ColumnarToRow (69) + +- CometFilter (68) + +- CometScan parquet spark_catalog.default.item (67) (unknown) Scan parquet spark_catalog.default.store_sales @@ -125,7 +127,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(11) ReusedExchange [Reuses operator id: 82] +(11) ReusedExchange [Reuses operator id: 84] Output [1]: [d_date_sk#14] (12) BroadcastHashJoin [codegen id : 5] @@ -208,49 +210,54 @@ Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#3 Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] Arguments: hashpartitioning(i_item_id#18, s_state#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometHashAggregate +(29) ColumnarToRow [codegen id : 6] +Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] + +(30) HashAggregate [codegen id : 6] Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] Keys [2]: [i_item_id#18, s_state#16] Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] +Aggregate Attributes [4]: [avg(agg1#19)#39, avg(UnscaledValue(agg2#20))#40, avg(UnscaledValue(agg3#21))#41, avg(UnscaledValue(agg4#22))#42] +Results [7]: [i_item_id#18, s_state#16, 0 AS g_state#43, avg(agg1#19)#39 AS agg1#44, cast((avg(UnscaledValue(agg2#20))#40 / 100.0) as decimal(11,6)) AS agg2#45, cast((avg(UnscaledValue(agg3#21))#41 / 100.0) as decimal(11,6)) AS agg3#46, cast((avg(UnscaledValue(agg4#22))#42 / 100.0) as decimal(11,6)) AS agg4#47] (unknown) Scan parquet spark_catalog.default.store_sales Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#48)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) -(32) ColumnarToRow [codegen id : 10] +(33) ColumnarToRow [codegen id : 11] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(33) ReusedExchange [Reuses operator id: 8] +(34) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#10] -(34) BroadcastHashJoin [codegen id : 10] +(35) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(35) Project [codegen id : 10] +(36) Project [codegen id : 11] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(36) ReusedExchange [Reuses operator id: 82] +(37) ReusedExchange [Reuses operator id: 84] Output [1]: [d_date_sk#14] -(37) BroadcastHashJoin [codegen id : 10] +(38) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(39) Project [codegen id : 11] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -261,114 +268,119 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(40) CometFilter +(41) CometFilter Input [2]: [s_store_sk#15, s_state#16] Condition : ((isnotnull(s_state#16) AND (s_state#16 = TN)) AND isnotnull(s_store_sk#15)) -(41) CometProject +(42) CometProject Input [2]: [s_store_sk#15, s_state#16] Arguments: [s_store_sk#15], [s_store_sk#15] -(42) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 9] Input [1]: [s_store_sk#15] -(43) BroadcastExchange +(44) BroadcastExchange Input [1]: [s_store_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(44) BroadcastHashJoin [codegen id : 10] +(45) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(45) Project [codegen id : 10] +(46) Project [codegen id : 11] Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15] -(46) ReusedExchange [Reuses operator id: 23] +(47) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#17, i_item_id#18] -(47) BroadcastHashJoin [codegen id : 10] +(48) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(48) Project [codegen id : 10] +(49) Project [codegen id : 11] Output [5]: [i_item_id#18, ss_quantity#4 AS agg1#19, ss_list_price#5 AS agg2#20, ss_coupon_amt#7 AS agg3#21, ss_sales_price#6 AS agg4#22] Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#17, i_item_id#18] -(49) HashAggregate [codegen id : 10] +(50) HashAggregate [codegen id : 11] Input [5]: [i_item_id#18, agg1#19, agg2#20, agg3#21, agg4#22] Keys [1]: [i_item_id#18] Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), partial_avg(UnscaledValue(agg3#21)), partial_avg(UnscaledValue(agg4#22))] -Aggregate Attributes [8]: [sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47] -Results [9]: [i_item_id#18, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +Aggregate Attributes [8]: [sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56] +Results [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] -(50) RowToColumnar -Input [9]: [i_item_id#18, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +(51) RowToColumnar +Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] -(51) CometColumnarExchange -Input [9]: [i_item_id#18, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +(52) CometColumnarExchange +Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(52) CometHashAggregate -Input [9]: [i_item_id#18, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +(53) ColumnarToRow [codegen id : 12] +Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] + +(54) HashAggregate [codegen id : 12] +Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] Keys [1]: [i_item_id#18] Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] +Aggregate Attributes [4]: [avg(agg1#19)#65, avg(UnscaledValue(agg2#20))#66, avg(UnscaledValue(agg3#21))#67, avg(UnscaledValue(agg4#22))#68] +Results [7]: [i_item_id#18, null AS s_state#69, 1 AS g_state#70, avg(agg1#19)#65 AS agg1#71, cast((avg(UnscaledValue(agg2#20))#66 / 100.0) as decimal(11,6)) AS agg2#72, cast((avg(UnscaledValue(agg3#21))#67 / 100.0) as decimal(11,6)) AS agg3#73, cast((avg(UnscaledValue(agg4#22))#68 / 100.0) as decimal(11,6)) AS agg4#74] (unknown) Scan parquet spark_catalog.default.store_sales Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#56)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#75)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(54) CometFilter +(56) CometFilter Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) -(55) ColumnarToRow [codegen id : 15] +(57) ColumnarToRow [codegen id : 17] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(56) ReusedExchange [Reuses operator id: 8] +(58) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#10] -(57) BroadcastHashJoin [codegen id : 15] +(59) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(58) Project [codegen id : 15] +(60) Project [codegen id : 17] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(59) ReusedExchange [Reuses operator id: 82] +(61) ReusedExchange [Reuses operator id: 84] Output [1]: [d_date_sk#14] -(60) BroadcastHashJoin [codegen id : 15] +(62) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(61) Project [codegen id : 15] +(63) Project [codegen id : 17] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -(62) ReusedExchange [Reuses operator id: 43] +(64) ReusedExchange [Reuses operator id: 44] Output [1]: [s_store_sk#15] -(63) BroadcastHashJoin [codegen id : 15] +(65) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(64) Project [codegen id : 15] +(66) Project [codegen id : 17] Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15] @@ -379,92 +391,91 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(66) CometFilter +(68) CometFilter Input [1]: [i_item_sk#17] Condition : isnotnull(i_item_sk#17) -(67) ColumnarToRow [codegen id : 14] +(69) ColumnarToRow [codegen id : 16] Input [1]: [i_item_sk#17] -(68) BroadcastExchange +(70) BroadcastExchange Input [1]: [i_item_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(69) BroadcastHashJoin [codegen id : 15] +(71) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(70) Project [codegen id : 15] +(72) Project [codegen id : 17] Output [4]: [ss_quantity#4 AS agg1#19, ss_list_price#5 AS agg2#20, ss_coupon_amt#7 AS agg3#21, ss_sales_price#6 AS agg4#22] Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#17] -(71) HashAggregate [codegen id : 15] +(73) HashAggregate [codegen id : 17] Input [4]: [agg1#19, agg2#20, agg3#21, agg4#22] Keys: [] Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), partial_avg(UnscaledValue(agg3#21)), partial_avg(UnscaledValue(agg4#22))] -Aggregate Attributes [8]: [sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] -Results [8]: [sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72] +Aggregate Attributes [8]: [sum#76, count#77, sum#78, count#79, sum#80, count#81, sum#82, count#83] +Results [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -(72) RowToColumnar -Input [8]: [sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72] +(74) RowToColumnar +Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -(73) CometColumnarExchange -Input [8]: [sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72] +(75) CometColumnarExchange +Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(74) CometHashAggregate -Input [8]: [sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72] +(76) ColumnarToRow [codegen id : 18] +Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] + +(77) HashAggregate [codegen id : 18] +Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] Keys: [] Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] +Aggregate Attributes [4]: [avg(agg1#19)#92, avg(UnscaledValue(agg2#20))#93, avg(UnscaledValue(agg3#21))#94, avg(UnscaledValue(agg4#22))#95] +Results [7]: [null AS i_item_id#96, null AS s_state#97, 1 AS g_state#98, avg(agg1#19)#92 AS agg1#99, cast((avg(UnscaledValue(agg2#20))#93 / 100.0) as decimal(11,6)) AS agg2#100, cast((avg(UnscaledValue(agg3#21))#94 / 100.0) as decimal(11,6)) AS agg3#101, cast((avg(UnscaledValue(agg4#22))#95 / 100.0) as decimal(11,6)) AS agg4#102] -(75) CometUnion -Child 0 Input [7]: [i_item_id#18, s_state#16, g_state#73, agg1#74, agg2#75, agg3#76, agg4#77] -Child 1 Input [7]: [i_item_id#18, s_state#78, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83] -Child 2 Input [7]: [i_item_id#84, s_state#85, g_state#86, agg1#87, agg2#88, agg3#89, agg4#90] - -(76) CometTakeOrderedAndProject -Input [7]: [i_item_id#18, s_state#16, g_state#73, agg1#74, agg2#75, agg3#76, agg4#77] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST,s_state#16 ASC NULLS FIRST], output=[i_item_id#18,s_state#16,g_state#73,agg1#74,agg2#75,agg3#76,agg4#77]), 100, [i_item_id#18 ASC NULLS FIRST, s_state#16 ASC NULLS FIRST], [i_item_id#18, s_state#16, g_state#73, agg1#74, agg2#75, agg3#76, agg4#77] +(78) Union -(77) ColumnarToRow [codegen id : 16] -Input [7]: [i_item_id#18, s_state#16, g_state#73, agg1#74, agg2#75, agg3#76, agg4#77] +(79) TakeOrderedAndProject +Input [7]: [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] +Arguments: 100, [i_item_id#18 ASC NULLS FIRST, s_state#16 ASC NULLS FIRST], [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (82) -+- * ColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (84) ++- * ColumnarToRow (83) + +- CometProject (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#91] +Output [2]: [d_date_sk#14, d_year#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(79) CometFilter -Input [2]: [d_date_sk#14, d_year#91] -Condition : ((isnotnull(d_year#91) AND (d_year#91 = 1998)) AND isnotnull(d_date_sk#14)) +(81) CometFilter +Input [2]: [d_date_sk#14, d_year#103] +Condition : ((isnotnull(d_year#103) AND (d_year#103 = 1998)) AND isnotnull(d_date_sk#14)) -(80) CometProject -Input [2]: [d_date_sk#14, d_year#91] +(82) CometProject +Input [2]: [d_date_sk#14, d_year#103] Arguments: [d_date_sk#14], [d_date_sk#14] -(81) ColumnarToRow [codegen id : 1] +(83) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(82) BroadcastExchange +(84) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt index fec1c27dd3..3a64f1517d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometUnion - CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] +TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id,s_state] #1 RowToColumnar WholeStageCodegen (5) @@ -52,10 +52,13 @@ WholeStageCodegen (16) InputAdapter CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange [i_item_id] #6 RowToColumnar - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -76,7 +79,7 @@ WholeStageCodegen (16) ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometProject [s_store_sk] @@ -84,10 +87,13 @@ WholeStageCodegen (16) CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] + WholeStageCodegen (18) + HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + ColumnarToRow + InputAdapter CometColumnarExchange #8 RowToColumnar - WholeStageCodegen (15) + WholeStageCodegen (17) HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -110,7 +116,7 @@ WholeStageCodegen (16) ReusedExchange [s_store_sk] #7 InputAdapter BroadcastExchange #9 - WholeStageCodegen (14) + WholeStageCodegen (16) ColumnarToRow InputAdapter CometFilter [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt index cd5187ddd2..b80a05efb2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt @@ -5,9 +5,9 @@ +- RowToColumnar (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) - :- * ColumnarToRow (26) - : +- CometFilter (25) - : +- CometHashAggregate (24) + :- * Filter (26) + : +- * HashAggregate (25) + : +- * ColumnarToRow (24) : +- CometColumnarExchange (23) : +- RowToColumnar (22) : +- * HashAggregate (21) @@ -143,59 +143,61 @@ Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometHashAggregate +(24) ColumnarToRow [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] + +(25) HashAggregate [codegen id : 6] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#16] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] -Condition : ((cnt#16 >= 15) AND (cnt#16 <= 20)) - -(26) ColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16] +(26) Filter [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) (unknown) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (28) CometFilter -Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] -Condition : isnotnull(c_customer_sk#17) +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Condition : isnotnull(c_customer_sk#18) (29) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] (30) BroadcastExchange -Input [5]: [c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#17] +Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None (32) Project [codegen id : 6] -Output [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#16, c_customer_sk#17, c_salutation#18, c_first_name#19, c_last_name#20, c_preferred_cust_flag#21] +Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] (33) RowToColumnar -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] (34) CometColumnarExchange -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] -Arguments: rangepartitioning(c_last_name#20 ASC NULLS FIRST, c_first_name#19 ASC NULLS FIRST, c_salutation#18 ASC NULLS FIRST, c_preferred_cust_flag#21 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] (35) CometSort -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] -Arguments: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16], [c_last_name#20 ASC NULLS FIRST, c_first_name#19 ASC NULLS FIRST, c_salutation#18 ASC NULLS FIRST, c_preferred_cust_flag#21 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] (36) ColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#20, c_first_name#19, c_salutation#18, c_preferred_cust_flag#21, ss_ticket_number#4, cnt#16] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] ===== Subqueries ===== @@ -208,18 +210,18 @@ BroadcastExchange (41) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#22, d_dom#23] +Output [3]: [d_date_sk#7, d_year#23, d_dom#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct (38) CometFilter -Input [3]: [d_date_sk#7, d_year#22, d_dom#23] -Condition : (((((d_dom#23 >= 1) AND (d_dom#23 <= 3)) OR ((d_dom#23 >= 25) AND (d_dom#23 <= 28))) AND d_year#22 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) (39) CometProject -Input [3]: [d_date_sk#7, d_year#22, d_dom#23] +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] Arguments: [d_date_sk#7], [d_date_sk#7] (40) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index 9c94a51d26..9fb9e6a631 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -7,10 +7,10 @@ WholeStageCodegen (7) WholeStageCodegen (6) Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 RowToColumnar WholeStageCodegen (4) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index 54878fd686..a0653a4981 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- * ColumnarToRow (42) +- CometColumnarExchange (41) +- RowToColumnar (40) +- * HashAggregate (39) @@ -236,17 +236,19 @@ Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, c Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(42) CometHashAggregate +(42) ColumnarToRow [codegen id : 10] +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] + +(43) HashAggregate [codegen id : 10] Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] +Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] +Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] -(43) CometTakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#19 ASC NULLS FIRST,cd_gender#21 ASC NULLS FIRST,cd_marital_status#22 ASC NULLS FIRST,cd_dep_count#23 ASC NULLS FIRST,cd_dep_employed_count#24 ASC NULLS FIRST,cd_dep_college_count#25 ASC NULLS FIRST], output=[ca_state#19,cd_gender#21,cd_marital_status#22,cd_dep_count#23,cnt1#52,avg(cd_dep_count)#53,max(cd_dep_count)#54,sum(cd_dep_count)#55,cd_dep_employed_count#24,cnt2#56,avg(cd_dep_employed_count)#57,max(cd_dep_employed_count)#58,sum(cd_dep_employed_count)#59,cd_dep_college_count#25,cnt3#60,avg(cd_dep_college_count)#61,max(cd_dep_college_count)#62,sum(cd_dep_college_count)#63]), 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] - -(44) ColumnarToRow [codegen id : 10] -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] +(44) TakeOrderedAndProject +Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] +Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] ===== Subqueries ===== @@ -259,18 +261,18 @@ BroadcastExchange (49) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#64, d_qoy#65] +Output [3]: [d_date_sk#9, d_year#74, d_qoy#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct (46) CometFilter -Input [3]: [d_date_sk#9, d_year#64, d_qoy#65] -Condition : ((((isnotnull(d_year#64) AND isnotnull(d_qoy#65)) AND (d_year#64 = 2002)) AND (d_qoy#65 < 4)) AND isnotnull(d_date_sk#9)) +Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) (47) CometProject -Input [3]: [d_date_sk#9, d_year#64, d_qoy#65] +Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] Arguments: [d_date_sk#9], [d_date_sk#9] (48) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index 549322409a..7e3ae715aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + WholeStageCodegen (10) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 RowToColumnar WholeStageCodegen (9) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index d12bef6d51..d9e550f2fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* ColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometHashAggregate (40) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- * ColumnarToRow (40) +- CometColumnarExchange (39) +- RowToColumnar (38) +- * HashAggregate (37) @@ -222,17 +222,19 @@ Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, c Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometHashAggregate +(40) ColumnarToRow [codegen id : 10] +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] + +(41) HashAggregate [codegen id : 10] Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] +Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] +Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] -(41) CometTakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#19 ASC NULLS FIRST,cd_gender#21 ASC NULLS FIRST,cd_marital_status#22 ASC NULLS FIRST,cd_dep_count#23 ASC NULLS FIRST,cd_dep_employed_count#24 ASC NULLS FIRST,cd_dep_college_count#25 ASC NULLS FIRST], output=[ca_state#19,cd_gender#21,cd_marital_status#22,cd_dep_count#23,cnt1#52,avg(cd_dep_count)#53,max(cd_dep_count)#54,sum(cd_dep_count)#55,cd_dep_employed_count#24,cnt2#56,avg(cd_dep_employed_count)#57,max(cd_dep_employed_count)#58,sum(cd_dep_employed_count)#59,cd_dep_college_count#25,cnt3#60,avg(cd_dep_college_count)#61,max(cd_dep_college_count)#62,sum(cd_dep_college_count)#63]), 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] - -(42) ColumnarToRow [codegen id : 10] -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#52, avg(cd_dep_count)#53, max(cd_dep_count)#54, sum(cd_dep_count)#55, cd_dep_employed_count#24, cnt2#56, avg(cd_dep_employed_count)#57, max(cd_dep_employed_count)#58, sum(cd_dep_employed_count)#59, cd_dep_college_count#25, cnt3#60, avg(cd_dep_college_count)#61, max(cd_dep_college_count)#62, sum(cd_dep_college_count)#63] +(42) TakeOrderedAndProject +Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] +Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] ===== Subqueries ===== @@ -245,18 +247,18 @@ BroadcastExchange (47) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#64, d_qoy#65] +Output [3]: [d_date_sk#7, d_year#74, d_qoy#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct (44) CometFilter -Input [3]: [d_date_sk#7, d_year#64, d_qoy#65] -Condition : ((((isnotnull(d_year#64) AND isnotnull(d_qoy#65)) AND (d_year#64 = 1999)) AND (d_qoy#65 < 4)) AND isnotnull(d_date_sk#7)) +Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] +Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 1999)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#7)) (45) CometProject -Input [3]: [d_date_sk#7, d_year#64, d_qoy#65] +Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] Arguments: [d_date_sk#7], [d_date_sk#7] (46) ColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index d5346d83b3..524a6ed84e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + WholeStageCodegen (10) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 RowToColumnar WholeStageCodegen (9) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt index 060abaefb8..a87fa1adc0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt @@ -1,47 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * ColumnarToRow (40) - +- CometSort (39) - +- CometColumnarExchange (38) - +- CometHashAggregate (37) - +- CometColumnarExchange (36) - +- CometHashAggregate (35) - +- CometUnion (34) - :- CometHashAggregate (23) - : +- CometColumnarExchange (22) - : +- RowToColumnar (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.store (13) - :- CometHashAggregate (28) - : +- CometColumnarExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - +- CometHashAggregate (33) - +- CometColumnarExchange (32) - +- CometHashAggregate (31) - +- CometHashAggregate (30) - +- ReusedExchange (29) +TakeOrderedAndProject (51) ++- * Project (50) + +- Window (49) + +- * ColumnarToRow (48) + +- CometSort (47) + +- CometColumnarExchange (46) + +- CometHashAggregate (45) + +- CometColumnarExchange (44) + +- RowToColumnar (43) + +- * HashAggregate (42) + +- Union (41) + :- * HashAggregate (24) + : +- * ColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- RowToColumnar (21) + : +- * HashAggregate (20) + : +- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.item (7) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.store (13) + :- * HashAggregate (32) + : +- * ColumnarToRow (31) + : +- CometColumnarExchange (30) + : +- RowToColumnar (29) + : +- * HashAggregate (28) + : +- * HashAggregate (27) + : +- * ColumnarToRow (26) + : +- ReusedExchange (25) + +- * HashAggregate (40) + +- * ColumnarToRow (39) + +- CometColumnarExchange (38) + +- RowToColumnar (37) + +- * HashAggregate (36) + +- * HashAggregate (35) + +- * ColumnarToRow (34) + +- ReusedExchange (33) (unknown) Scan parquet spark_catalog.default.store_sales @@ -59,7 +67,7 @@ Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 48] +(4) ReusedExchange [Reuses operator id: 56] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -146,126 +154,163 @@ Input [4]: [i_category#10, i_class#9, sum#15, sum#16] Input [4]: [i_category#10, i_class#9, sum#15, sum#16] Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometHashAggregate +(23) ColumnarToRow [codegen id : 5] +Input [4]: [i_category#10, i_class#9, sum#15, sum#16] + +(24) HashAggregate [codegen id : 5] Input [4]: [i_category#10, i_class#9, sum#15, sum#16] Keys [2]: [i_category#10, i_class#9] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#17, sum(UnscaledValue(ss_ext_sales_price#3))#18] +Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#17,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2)) as decimal(38,20)) AS gross_margin#19, i_category#10, i_class#9, 0 AS t_category#20, 0 AS t_class#21, 0 AS lochierarchy#22] + +(25) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#10, i_class#9, sum#23, sum#24] -(24) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#10, i_class#9, sum#17, sum#18] +(26) ColumnarToRow [codegen id : 10] +Input [4]: [i_category#10, i_class#9, sum#23, sum#24] -(25) CometHashAggregate -Input [4]: [i_category#10, i_class#9, sum#17, sum#18] +(27) HashAggregate [codegen id : 10] +Input [4]: [i_category#10, i_class#9, sum#23, sum#24] Keys [2]: [i_category#10, i_class#9] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#25, sum(UnscaledValue(ss_ext_sales_price#3))#26] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#25,17,2) AS ss_net_profit#27, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#26,17,2) AS ss_ext_sales_price#28, i_category#10] -(26) CometHashAggregate -Input [3]: [ss_net_profit#19, ss_ext_sales_price#20, i_category#10] +(28) HashAggregate [codegen id : 10] +Input [3]: [ss_net_profit#27, ss_ext_sales_price#28, i_category#10] Keys [1]: [i_category#10] -Functions [2]: [partial_sum(ss_net_profit#19), partial_sum(ss_ext_sales_price#20)] +Functions [2]: [partial_sum(ss_net_profit#27), partial_sum(ss_ext_sales_price#28)] +Aggregate Attributes [4]: [sum#29, isEmpty#30, sum#31, isEmpty#32] +Results [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] -(27) CometColumnarExchange -Input [5]: [i_category#10, sum#21, isEmpty#22, sum#23, isEmpty#24] +(29) RowToColumnar +Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] + +(30) CometColumnarExchange +Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) CometHashAggregate -Input [5]: [i_category#10, sum#21, isEmpty#22, sum#23, isEmpty#24] +(31) ColumnarToRow [codegen id : 11] +Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] + +(32) HashAggregate [codegen id : 11] +Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] Keys [1]: [i_category#10] -Functions [2]: [sum(ss_net_profit#19), sum(ss_ext_sales_price#20)] +Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)] +Aggregate Attributes [2]: [sum(ss_net_profit#27)#37, sum(ss_ext_sales_price#28)#38] +Results [6]: [cast((sum(ss_net_profit#27)#37 / sum(ss_ext_sales_price#28)#38) as decimal(38,20)) AS gross_margin#39, i_category#10, null AS i_class#40, 0 AS t_category#41, 1 AS t_class#42, 1 AS lochierarchy#43] + +(33) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#10, i_class#9, sum#44, sum#45] -(29) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#10, i_class#9, sum#25, sum#26] +(34) ColumnarToRow [codegen id : 16] +Input [4]: [i_category#10, i_class#9, sum#44, sum#45] -(30) CometHashAggregate -Input [4]: [i_category#10, i_class#9, sum#25, sum#26] +(35) HashAggregate [codegen id : 16] +Input [4]: [i_category#10, i_class#9, sum#44, sum#45] Keys [2]: [i_category#10, i_class#9] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#25, sum(UnscaledValue(ss_ext_sales_price#3))#26] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#25,17,2) AS ss_net_profit#27, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#26,17,2) AS ss_ext_sales_price#28] -(31) CometHashAggregate -Input [2]: [ss_net_profit#19, ss_ext_sales_price#20] +(36) HashAggregate [codegen id : 16] +Input [2]: [ss_net_profit#27, ss_ext_sales_price#28] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#19), partial_sum(ss_ext_sales_price#20)] +Functions [2]: [partial_sum(ss_net_profit#27), partial_sum(ss_ext_sales_price#28)] +Aggregate Attributes [4]: [sum#46, isEmpty#47, sum#48, isEmpty#49] +Results [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] -(32) CometColumnarExchange -Input [4]: [sum#27, isEmpty#28, sum#29, isEmpty#30] +(37) RowToColumnar +Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] + +(38) CometColumnarExchange +Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(33) CometHashAggregate -Input [4]: [sum#27, isEmpty#28, sum#29, isEmpty#30] +(39) ColumnarToRow [codegen id : 17] +Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] + +(40) HashAggregate [codegen id : 17] +Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] Keys: [] -Functions [2]: [sum(ss_net_profit#19), sum(ss_ext_sales_price#20)] +Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)] +Aggregate Attributes [2]: [sum(ss_net_profit#27)#54, sum(ss_ext_sales_price#28)#55] +Results [6]: [cast((sum(ss_net_profit#27)#54 / sum(ss_ext_sales_price#28)#55) as decimal(38,20)) AS gross_margin#56, null AS i_category#57, null AS i_class#58, 1 AS t_category#59, 1 AS t_class#60, 2 AS lochierarchy#61] -(34) CometUnion -Child 0 Input [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] -Child 1 Input [6]: [gross_margin#35, i_category#10, i_class#36, t_category#37, t_class#38, lochierarchy#39] -Child 2 Input [6]: [gross_margin#40, i_category#41, i_class#42, t_category#43, t_class#44, lochierarchy#45] +(41) Union -(35) CometHashAggregate -Input [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] -Keys [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] +(42) HashAggregate [codegen id : 18] +Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] Functions: [] +Aggregate Attributes: [] +Results [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -(36) CometColumnarExchange -Input [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] -Arguments: hashpartitioning(gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(43) RowToColumnar +Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -(37) CometHashAggregate -Input [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] -Keys [6]: [gross_margin#31, i_category#10, i_class#9, t_category#32, t_class#33, lochierarchy#34] +(44) CometColumnarExchange +Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +Arguments: hashpartitioning(gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(45) CometHashAggregate +Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] Functions: [] -(38) CometColumnarExchange -Input [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46] -Arguments: hashpartitioning(lochierarchy#34, _w0#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(46) CometColumnarExchange +Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] +Arguments: hashpartitioning(lochierarchy#22, _w0#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(39) CometSort -Input [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46] -Arguments: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46], [lochierarchy#34 ASC NULLS FIRST, _w0#46 ASC NULLS FIRST, gross_margin#31 ASC NULLS FIRST] +(47) CometSort +Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] +Arguments: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62], [lochierarchy#22 ASC NULLS FIRST, _w0#62 ASC NULLS FIRST, gross_margin#19 ASC NULLS FIRST] -(40) ColumnarToRow [codegen id : 13] -Input [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46] +(48) ColumnarToRow [codegen id : 19] +Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] -(41) Window -Input [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46] -Arguments: [rank(gross_margin#31) windowspecdefinition(lochierarchy#34, _w0#46, gross_margin#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#47], [lochierarchy#34, _w0#46], [gross_margin#31 ASC NULLS FIRST] +(49) Window +Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] +Arguments: [rank(gross_margin#19) windowspecdefinition(lochierarchy#22, _w0#62, gross_margin#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#63], [lochierarchy#22, _w0#62], [gross_margin#19 ASC NULLS FIRST] -(42) Project [codegen id : 14] -Output [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, rank_within_parent#47] -Input [6]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, _w0#46, rank_within_parent#47] +(50) Project [codegen id : 20] +Output [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] +Input [6]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62, rank_within_parent#63] -(43) TakeOrderedAndProject -Input [5]: [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, rank_within_parent#47] -Arguments: 100, [lochierarchy#34 DESC NULLS LAST, CASE WHEN (lochierarchy#34 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [gross_margin#31, i_category#10, i_class#9, lochierarchy#34, rank_within_parent#47] +(51) TakeOrderedAndProject +Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] +Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (56) ++- * ColumnarToRow (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#48] +Output [2]: [d_date_sk#7, d_year#64] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [2]: [d_date_sk#7, d_year#48] -Condition : ((isnotnull(d_year#48) AND (d_year#48 = 2001)) AND isnotnull(d_date_sk#7)) +(53) CometFilter +Input [2]: [d_date_sk#7, d_year#64] +Condition : ((isnotnull(d_year#64) AND (d_year#64 = 2001)) AND isnotnull(d_date_sk#7)) -(46) CometProject -Input [2]: [d_date_sk#7, d_year#48] +(54) CometProject +Input [2]: [d_date_sk#7, d_year#64] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) ColumnarToRow [codegen id : 1] +(55) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(48) BroadcastExchange +(56) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index 29ae8ca601..c7fb8c2303 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -1,64 +1,84 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (14) + WholeStageCodegen (20) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (13) + WholeStageCodegen (19) ColumnarToRow InputAdapter CometSort [lochierarchy,_w0,gross_margin] CometColumnarExchange [lochierarchy,_w0] #1 CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion - CometHashAggregate [i_category,i_class,sum,sum] - CometColumnarExchange [i_category,i_class] #3 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) + RowToColumnar + WholeStageCodegen (18) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,t_category,t_class,lochierarchy,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #3 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (11) + HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #7 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange #8 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [i_category,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [i_category] #7 - CometHashAggregate [i_category,ss_net_profit,ss_ext_sales_price] - CometHashAggregate [i_category,i_class,sum,sum] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] - CometColumnarExchange #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] - CometHashAggregate [i_category,i_class,sum,sum] - ReusedExchange [i_category,i_class,sum,sum] #3 + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index aa410a6b55..109830f2ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -1,51 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * ColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- CometHashAggregate (22) - : : +- CometColumnarExchange (21) - : : +- RowToColumnar (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- Window (34) - : +- * ColumnarToRow (33) - : +- CometSort (32) - : +- ReusedExchange (31) - +- BroadcastExchange (44) - +- * Project (43) - +- Window (42) - +- * ColumnarToRow (41) - +- CometSort (40) - +- ReusedExchange (39) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (32) + : : +- * Filter (31) + : : +- Window (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * ColumnarToRow (27) + : : +- CometSort (26) + : : +- CometColumnarExchange (25) + : : +- RowToColumnar (24) + : : +- * HashAggregate (23) + : : +- * ColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- BroadcastExchange (42) + : +- * Project (41) + : +- Window (40) + : +- * ColumnarToRow (39) + : +- CometSort (38) + : +- CometColumnarExchange (37) + : +- RowToColumnar (36) + : +- * HashAggregate (35) + : +- * ColumnarToRow (34) + : +- ReusedExchange (33) + +- BroadcastExchange (50) + +- * Project (49) + +- Window (48) + +- * ColumnarToRow (47) + +- CometSort (46) + +- ReusedExchange (45) (unknown) Scan parquet spark_catalog.default.item @@ -91,7 +97,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 51] +(10) ReusedExchange [Reuses operator id: 57] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -146,117 +152,142 @@ Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year# Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometHashAggregate +(22) ColumnarToRow [codegen id : 5] +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] + +(23) HashAggregate [codegen id : 5] Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#17] +Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS _w0#19] + +(24) RowToColumnar +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -(23) CometColumnarExchange -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +(25) CometColumnarExchange +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometSort -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(26) CometSort +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 5] -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +(27) ColumnarToRow [codegen id : 6] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -(26) Window -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(28) Window +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(27) Filter [codegen id : 6] -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +(29) Filter [codegen id : 7] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(28) Window -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] +(30) Window +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] + +(31) Filter [codegen id : 22] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) + +(32) Project [codegen id : 22] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] + +(33) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] + +(34) ColumnarToRow [codegen id : 12] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(29) Filter [codegen id : 19] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) +(35) HashAggregate [codegen id : 12] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] +Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] +Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] -(30) Project [codegen id : 19] -Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +(36) RowToColumnar +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -(31) ReusedExchange [Reuses operator id: 23] -Output [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +(37) CometColumnarExchange +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) CometSort -Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] -Arguments: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, s_store_name#23 ASC NULLS FIRST, s_company_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(38) CometSort +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 11] -Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +(39) ColumnarToRow [codegen id : 13] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -(34) Window -Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] -Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#27], [i_category#21, i_brand#22, s_store_name#23, s_company_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(40) Window +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(35) Project [codegen id : 12] -Output [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#17 AS sum_sales#28, rn#27] -Input [8]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17, rn#27] +(41) Project [codegen id : 14] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] -(36) BroadcastExchange -Input [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#28, rn#27] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=5] +(42) BroadcastExchange +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 19] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] -Right keys [5]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, (rn#27 + 1)] +(43) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] +Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] Join type: Inner Join condition: None -(38) Project [codegen id : 19] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] -Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#28, rn#27] +(44) Project [codegen id : 22] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -(39) ReusedExchange [Reuses operator id: 23] -Output [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] +(45) ReusedExchange [Reuses operator id: 37] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -(40) CometSort -Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17], [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, s_store_name#31 ASC NULLS FIRST, s_company_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +(46) CometSort +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 17] -Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] +(47) ColumnarToRow [codegen id : 20] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -(42) Window -Input [7]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#29, i_brand#30, s_store_name#31, s_company_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +(48) Window +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(43) Project [codegen id : 18] -Output [6]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#17 AS sum_sales#36, rn#35] -Input [8]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] +(49) Project [codegen id : 21] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] -(44) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#36, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=6] +(50) BroadcastExchange +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(45) BroadcastHashJoin [codegen id : 19] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#31, s_company_name#32, (rn#35 - 1)] +(51) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] +Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] Join type: Inner Join condition: None -(46) Project [codegen id : 19] -Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#36 AS nsum#38] -Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#29, i_brand#30, s_store_name#31, s_company_name#32, sum_sales#36, rn#35] +(52) Project [codegen id : 22] +Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -(47) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +(53) TakeOrderedAndProject +Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (51) -+- * ColumnarToRow (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) (unknown) Scan parquet spark_catalog.default.date_dim @@ -266,15 +297,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter +(55) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(51) BroadcastExchange +(57) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index d3aeefb861..0ea08b4352 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (19) + WholeStageCodegen (22) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,72 +8,82 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (6) + WholeStageCodegen (7) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (12) + WholeStageCodegen (14) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (11) + WholeStageCodegen (13) ColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 + RowToColumnar + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (18) + BroadcastExchange #8 + WholeStageCodegen (21) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (17) + WholeStageCodegen (20) ColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #1 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index 7d3280c095..f2e628b732 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -1,86 +1,92 @@ == Physical Plan == -* ColumnarToRow (82) -+- CometTakeOrderedAndProject (81) - +- CometHashAggregate (80) - +- CometColumnarExchange (79) - +- RowToColumnar (78) - +- * HashAggregate (77) - +- Union (76) - :- * Project (25) - : +- * Filter (24) - : +- Window (23) - : +- * Sort (22) - : +- Window (21) - : +- * ColumnarToRow (20) - : +- CometSort (19) - : +- CometColumnarExchange (18) - : +- CometHashAggregate (17) - : +- CometColumnarExchange (16) - : +- RowToColumnar (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.web_returns (5) - : +- ReusedExchange (11) - :- * Project (50) - : +- * Filter (49) - : +- Window (48) - : +- * Sort (47) - : +- Window (46) - : +- * ColumnarToRow (45) - : +- CometSort (44) - : +- CometColumnarExchange (43) - : +- CometHashAggregate (42) - : +- CometColumnarExchange (41) - : +- RowToColumnar (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * ColumnarToRow (35) - : : +- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometBroadcastExchange (29) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (26) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (36) - +- * Project (75) - +- * Filter (74) - +- Window (73) - +- * Sort (72) - +- Window (71) - +- * ColumnarToRow (70) - +- CometSort (69) - +- CometColumnarExchange (68) - +- CometHashAggregate (67) - +- CometColumnarExchange (66) - +- RowToColumnar (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) - : +- CometProject (59) - : +- CometBroadcastHashJoin (58) - : :- CometBroadcastExchange (54) - : : +- CometProject (53) - : : +- CometFilter (52) - : : +- CometScan parquet spark_catalog.default.store_sales (51) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan parquet spark_catalog.default.store_returns (55) - +- ReusedExchange (61) +* ColumnarToRow (88) ++- CometTakeOrderedAndProject (87) + +- CometHashAggregate (86) + +- CometColumnarExchange (85) + +- RowToColumnar (84) + +- * HashAggregate (83) + +- Union (82) + :- * Project (27) + : +- * Filter (26) + : +- Window (25) + : +- * Sort (24) + : +- Window (23) + : +- * ColumnarToRow (22) + : +- CometSort (21) + : +- CometColumnarExchange (20) + : +- RowToColumnar (19) + : +- * HashAggregate (18) + : +- * ColumnarToRow (17) + : +- CometColumnarExchange (16) + : +- RowToColumnar (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- ReusedExchange (11) + :- * Project (54) + : +- * Filter (53) + : +- Window (52) + : +- * Sort (51) + : +- Window (50) + : +- * ColumnarToRow (49) + : +- CometSort (48) + : +- CometColumnarExchange (47) + : +- RowToColumnar (46) + : +- * HashAggregate (45) + : +- * ColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- RowToColumnar (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * ColumnarToRow (37) + : : +- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometBroadcastExchange (31) + : : : +- CometProject (30) + : : : +- CometFilter (29) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) + : : +- CometProject (34) + : : +- CometFilter (33) + : : +- CometScan parquet spark_catalog.default.catalog_returns (32) + : +- ReusedExchange (38) + +- * Project (81) + +- * Filter (80) + +- Window (79) + +- * Sort (78) + +- Window (77) + +- * ColumnarToRow (76) + +- CometSort (75) + +- CometColumnarExchange (74) + +- RowToColumnar (73) + +- * HashAggregate (72) + +- * ColumnarToRow (71) + +- CometColumnarExchange (70) + +- RowToColumnar (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * ColumnarToRow (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometBroadcastExchange (58) + : : +- CometProject (57) + : : +- CometFilter (56) + : : +- CometScan parquet spark_catalog.default.store_sales (55) + : +- CometProject (61) + : +- CometFilter (60) + : +- CometScan parquet spark_catalog.default.store_returns (59) + +- ReusedExchange (65) (unknown) Scan parquet spark_catalog.default.web_sales @@ -130,7 +136,7 @@ Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_re (10) ColumnarToRow [codegen id : 2] Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(11) ReusedExchange [Reuses operator id: 87] +(11) ReusedExchange [Reuses operator id: 93] Output [1]: [d_date_sk#13] (12) BroadcastHashJoin [codegen id : 2] @@ -157,324 +163,348 @@ Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25 Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(17) CometHashAggregate +(17) ColumnarToRow [codegen id : 3] +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] + +(18) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] +Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] + +(19) RowToColumnar +Input [3]: [item#30, return_ratio#31, currency_ratio#32] -(18) CometColumnarExchange -Input [3]: [item#26, return_ratio#27, currency_ratio#28] +(20) CometColumnarExchange +Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) CometSort -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: [item#26, return_ratio#27, currency_ratio#28], [return_ratio#27 ASC NULLS FIRST] +(21) CometSort +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] -(20) ColumnarToRow [codegen id : 3] -Input [3]: [item#26, return_ratio#27, currency_ratio#28] +(22) ColumnarToRow [codegen id : 4] +Input [3]: [item#30, return_ratio#31, currency_ratio#32] -(21) Window -Input [3]: [item#26, return_ratio#27, currency_ratio#28] -Arguments: [rank(return_ratio#27) windowspecdefinition(return_ratio#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#29], [return_ratio#27 ASC NULLS FIRST] +(23) Window +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(22) Sort [codegen id : 4] -Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] -Arguments: [currency_ratio#28 ASC NULLS FIRST], false, 0 +(24) Sort [codegen id : 5] +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 -(23) Window -Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] -Arguments: [rank(currency_ratio#28) windowspecdefinition(currency_ratio#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#30], [currency_ratio#28 ASC NULLS FIRST] +(25) Window +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(24) Filter [codegen id : 5] -Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] -Condition : ((return_rank#29 <= 10) OR (currency_rank#30 <= 10)) +(26) Filter [codegen id : 6] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(25) Project [codegen id : 5] -Output [5]: [web AS channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] +(27) Project [codegen id : 6] +Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] +Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(27) CometFilter -Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] -Condition : (((((((isnotnull(cs_net_profit#36) AND isnotnull(cs_net_paid#35)) AND isnotnull(cs_quantity#34)) AND (cs_net_profit#36 > 1.00)) AND (cs_net_paid#35 > 0.00)) AND (cs_quantity#34 > 0)) AND isnotnull(cs_order_number#33)) AND isnotnull(cs_item_sk#32)) +(29) CometFilter +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(28) CometProject -Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] -Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37], [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +(30) CometProject +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -(29) CometBroadcastExchange -Input [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] -Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +(31) CometBroadcastExchange +Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] +Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter -Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] -Condition : (((isnotnull(cr_return_amount#42) AND (cr_return_amount#42 > 10000.00)) AND isnotnull(cr_order_number#40)) AND isnotnull(cr_item_sk#39)) +(33) CometFilter +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) -(32) CometProject -Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] -Arguments: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42], [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] +(34) CometProject +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -(33) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] -Right output [4]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] -Arguments: [cs_order_number#33, cs_item_sk#32], [cr_order_number#40, cr_item_sk#39], Inner +(35) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner -(34) CometProject -Input [9]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] -Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] +(36) CometProject +Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -(35) ColumnarToRow [codegen id : 7] -Input [6]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] +(37) ColumnarToRow [codegen id : 8] +Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -(36) ReusedExchange [Reuses operator id: 87] -Output [1]: [d_date_sk#44] +(38) ReusedExchange [Reuses operator id: 93] +Output [1]: [d_date_sk#48] -(37) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#44] +(39) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#41] +Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(38) Project [codegen id : 7] -Output [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] -Input [7]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42, d_date_sk#44] +(40) Project [codegen id : 8] +Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] -(39) HashAggregate [codegen id : 7] -Input [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] -Keys [1]: [cs_item_sk#32] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#41, 0)), partial_sum(coalesce(cs_quantity#34, 0)), partial_sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Results [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] +(41) HashAggregate [codegen id : 8] +Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Keys [1]: [cs_item_sk#36] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] +Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -(40) RowToColumnar -Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] +(42) RowToColumnar +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -(41) CometColumnarExchange -Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] -Arguments: hashpartitioning(cs_item_sk#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(43) CometColumnarExchange +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(42) CometHashAggregate -Input [7]: [cs_item_sk#32, sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] -Keys [1]: [cs_item_sk#32] -Functions [4]: [sum(coalesce(cr_return_quantity#41, 0)), sum(coalesce(cs_quantity#34, 0)), sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] +(44) ColumnarToRow [codegen id : 9] +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -(43) CometColumnarExchange -Input [3]: [item#57, return_ratio#58, currency_ratio#59] +(45) HashAggregate [codegen id : 9] +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Keys [1]: [cs_item_sk#36] +Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] +Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] + +(46) RowToColumnar +Input [3]: [item#65, return_ratio#66, currency_ratio#67] + +(47) CometColumnarExchange +Input [3]: [item#65, return_ratio#66, currency_ratio#67] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) CometSort -Input [3]: [item#57, return_ratio#58, currency_ratio#59] -Arguments: [item#57, return_ratio#58, currency_ratio#59], [return_ratio#58 ASC NULLS FIRST] +(48) CometSort +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [item#65, return_ratio#66, currency_ratio#67], [return_ratio#66 ASC NULLS FIRST] -(45) ColumnarToRow [codegen id : 8] -Input [3]: [item#57, return_ratio#58, currency_ratio#59] +(49) ColumnarToRow [codegen id : 10] +Input [3]: [item#65, return_ratio#66, currency_ratio#67] -(46) Window -Input [3]: [item#57, return_ratio#58, currency_ratio#59] -Arguments: [rank(return_ratio#58) windowspecdefinition(return_ratio#58 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#60], [return_ratio#58 ASC NULLS FIRST] +(50) Window +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] -(47) Sort [codegen id : 9] -Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] -Arguments: [currency_ratio#59 ASC NULLS FIRST], false, 0 +(51) Sort [codegen id : 11] +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 -(48) Window -Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] -Arguments: [rank(currency_ratio#59) windowspecdefinition(currency_ratio#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#61], [currency_ratio#59 ASC NULLS FIRST] +(52) Window +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] -(49) Filter [codegen id : 10] -Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] -Condition : ((return_rank#60 <= 10) OR (currency_rank#61 <= 10)) +(53) Filter [codegen id : 12] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) -(50) Project [codegen id : 10] -Output [5]: [catalog AS channel#62, item#57, return_ratio#58, return_rank#60, currency_rank#61] -Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] +(54) Project [codegen id : 12] +Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] (unknown) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] +Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#68), dynamicpruningexpression(ss_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(52) CometFilter -Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] -Condition : (((((((isnotnull(ss_net_profit#67) AND isnotnull(ss_net_paid#66)) AND isnotnull(ss_quantity#65)) AND (ss_net_profit#67 > 1.00)) AND (ss_net_paid#66 > 0.00)) AND (ss_quantity#65 > 0)) AND isnotnull(ss_ticket_number#64)) AND isnotnull(ss_item_sk#63)) +(56) CometFilter +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) -(53) CometProject -Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] -Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68], [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +(57) CometProject +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -(54) CometBroadcastExchange -Input [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] -Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +(58) CometBroadcastExchange +Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] (unknown) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] +Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(56) CometFilter -Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] -Condition : (((isnotnull(sr_return_amt#73) AND (sr_return_amt#73 > 10000.00)) AND isnotnull(sr_ticket_number#71)) AND isnotnull(sr_item_sk#70)) +(60) CometFilter +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) -(57) CometProject -Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] -Arguments: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73], [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] +(61) CometProject +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -(58) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] -Right output [4]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] -Arguments: [ss_ticket_number#64, ss_item_sk#63], [sr_ticket_number#71, sr_item_sk#70], Inner +(62) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner -(59) CometProject -Input [9]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] -Arguments: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73], [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] +(63) CometProject +Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -(60) ColumnarToRow [codegen id : 12] -Input [6]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] +(64) ColumnarToRow [codegen id : 14] +Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -(61) ReusedExchange [Reuses operator id: 87] -Output [1]: [d_date_sk#75] +(65) ReusedExchange [Reuses operator id: 93] +Output [1]: [d_date_sk#83] -(62) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [ss_sold_date_sk#68] -Right keys [1]: [d_date_sk#75] +(66) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ss_sold_date_sk#76] +Right keys [1]: [d_date_sk#83] Join type: Inner Join condition: None -(63) Project [codegen id : 12] -Output [5]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] -Input [7]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73, d_date_sk#75] +(67) Project [codegen id : 14] +Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] + +(68) HashAggregate [codegen id : 14] +Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Keys [1]: [ss_item_sk#71] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] +Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] + +(69) RowToColumnar +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -(64) HashAggregate [codegen id : 12] -Input [5]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] -Keys [1]: [ss_item_sk#63] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#72, 0)), partial_sum(coalesce(ss_quantity#65, 0)), partial_sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#76, sum#77, sum#78, isEmpty#79, sum#80, isEmpty#81] -Results [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +(70) CometColumnarExchange +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(65) RowToColumnar -Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +(71) ColumnarToRow [codegen id : 15] +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -(66) CometColumnarExchange -Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Arguments: hashpartitioning(ss_item_sk#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(72) HashAggregate [codegen id : 15] +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Keys [1]: [ss_item_sk#71] +Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] +Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] -(67) CometHashAggregate -Input [7]: [ss_item_sk#63, sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [1]: [ss_item_sk#63] -Functions [4]: [sum(coalesce(sr_return_quantity#72, 0)), sum(coalesce(ss_quantity#65, 0)), sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] +(73) RowToColumnar +Input [3]: [item#100, return_ratio#101, currency_ratio#102] -(68) CometColumnarExchange -Input [3]: [item#88, return_ratio#89, currency_ratio#90] +(74) CometColumnarExchange +Input [3]: [item#100, return_ratio#101, currency_ratio#102] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(69) CometSort -Input [3]: [item#88, return_ratio#89, currency_ratio#90] -Arguments: [item#88, return_ratio#89, currency_ratio#90], [return_ratio#89 ASC NULLS FIRST] +(75) CometSort +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [item#100, return_ratio#101, currency_ratio#102], [return_ratio#101 ASC NULLS FIRST] -(70) ColumnarToRow [codegen id : 13] -Input [3]: [item#88, return_ratio#89, currency_ratio#90] +(76) ColumnarToRow [codegen id : 16] +Input [3]: [item#100, return_ratio#101, currency_ratio#102] -(71) Window -Input [3]: [item#88, return_ratio#89, currency_ratio#90] -Arguments: [rank(return_ratio#89) windowspecdefinition(return_ratio#89 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#91], [return_ratio#89 ASC NULLS FIRST] +(77) Window +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] -(72) Sort [codegen id : 14] -Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] -Arguments: [currency_ratio#90 ASC NULLS FIRST], false, 0 +(78) Sort [codegen id : 17] +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 -(73) Window -Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] -Arguments: [rank(currency_ratio#90) windowspecdefinition(currency_ratio#90 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#92], [currency_ratio#90 ASC NULLS FIRST] +(79) Window +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] -(74) Filter [codegen id : 15] -Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] -Condition : ((return_rank#91 <= 10) OR (currency_rank#92 <= 10)) +(80) Filter [codegen id : 18] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] +Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) -(75) Project [codegen id : 15] -Output [5]: [store AS channel#93, item#88, return_ratio#89, return_rank#91, currency_rank#92] -Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] +(81) Project [codegen id : 18] +Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -(76) Union +(82) Union -(77) HashAggregate [codegen id : 16] -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +(83) HashAggregate [codegen id : 19] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(78) RowToColumnar -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +(84) RowToColumnar +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(79) CometColumnarExchange -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -Arguments: hashpartitioning(channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(85) CometColumnarExchange +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(80) CometHashAggregate -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +(86) CometHashAggregate +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] -(81) CometTakeOrderedAndProject -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,return_rank#29 ASC NULLS FIRST,currency_rank#30 ASC NULLS FIRST,item#26 ASC NULLS FIRST], output=[channel#31,item#26,return_ratio#27,return_rank#29,currency_rank#30]), 100, [channel#31 ASC NULLS FIRST, return_rank#29 ASC NULLS FIRST, currency_rank#30 ASC NULLS FIRST, item#26 ASC NULLS FIRST], [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +(87) CometTakeOrderedAndProject +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(82) ColumnarToRow [codegen id : 17] -Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +(88) ColumnarToRow [codegen id : 20] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (87) -+- * ColumnarToRow (86) - +- CometProject (85) - +- CometFilter (84) - +- CometScan parquet spark_catalog.default.date_dim (83) +BroadcastExchange (93) ++- * ColumnarToRow (92) + +- CometProject (91) + +- CometFilter (90) + +- CometScan parquet spark_catalog.default.date_dim (89) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#94, d_moy#95] +Output [3]: [d_date_sk#13, d_year#106, d_moy#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(84) CometFilter -Input [3]: [d_date_sk#13, d_year#94, d_moy#95] -Condition : ((((isnotnull(d_year#94) AND isnotnull(d_moy#95)) AND (d_year#94 = 2001)) AND (d_moy#95 = 12)) AND isnotnull(d_date_sk#13)) +(90) CometFilter +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) -(85) CometProject -Input [3]: [d_date_sk#13, d_year#94, d_moy#95] +(91) CometProject +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Arguments: [d_date_sk#13], [d_date_sk#13] -(86) ColumnarToRow [codegen id : 1] +(92) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(87) BroadcastExchange +(93) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#68 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index 6ea9407566..42f84a6e7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -1,123 +1,135 @@ -WholeStageCodegen (17) +WholeStageCodegen (20) ColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 RowToColumnar - WholeStageCodegen (16) + WholeStageCodegen (19) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (6) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (4) + WholeStageCodegen (5) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (3) + WholeStageCodegen (4) ColumnarToRow InputAdapter CometSort [return_ratio] CometColumnarExchange #2 - CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [ws_item_sk] #3 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (10) + RowToColumnar + WholeStageCodegen (3) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk] #3 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (12) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (9) + WholeStageCodegen (11) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (8) + WholeStageCodegen (10) ColumnarToRow InputAdapter CometSort [return_ratio] CometColumnarExchange #6 - CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [cs_item_sk] #7 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #8 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (15) + RowToColumnar + WholeStageCodegen (9) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [cs_item_sk] #7 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #8 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (18) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (14) + WholeStageCodegen (17) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (13) + WholeStageCodegen (16) ColumnarToRow InputAdapter CometSort [return_ratio] CometColumnarExchange #9 - CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [ss_item_sk] #10 - RowToColumnar - WholeStageCodegen (12) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #11 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 + RowToColumnar + WholeStageCodegen (15) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk] #10 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #11 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index 60f983923a..3498539ba1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -1,81 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * Filter (76) - +- * HashAggregate (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- Window (65) - : +- * ColumnarToRow (64) - : +- CometSort (63) - : +- CometColumnarExchange (62) - : +- CometProject (61) - : +- CometFilter (60) - : +- CometSortMergeJoin (59) - : :- CometSort (29) - : : +- CometColumnarExchange (28) - : : +- CometHashAggregate (27) - : : +- CometColumnarExchange (26) - : : +- RowToColumnar (25) - : : +- * HashAggregate (24) - : : +- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (15) - : : : +- Window (14) - : : : +- * ColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- CometHashAggregate (10) - : : : +- CometColumnarExchange (9) - : : : +- RowToColumnar (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (21) - : : +- * Project (20) - : : +- Window (19) - : : +- * ColumnarToRow (18) - : : +- CometSort (17) - : : +- ReusedExchange (16) - : +- CometSort (58) - : +- CometColumnarExchange (57) - : +- CometHashAggregate (56) - : +- CometColumnarExchange (55) - : +- RowToColumnar (54) - : +- * HashAggregate (53) - : +- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (44) - : : +- Window (43) - : : +- * ColumnarToRow (42) - : : +- CometSort (41) - : : +- CometColumnarExchange (40) - : : +- CometHashAggregate (39) - : : +- CometColumnarExchange (38) - : : +- RowToColumnar (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * ColumnarToRow (32) - : : : +- CometFilter (31) - : : : +- CometScan parquet spark_catalog.default.store_sales (30) - : : +- ReusedExchange (33) - : +- BroadcastExchange (50) - : +- * Project (49) - : +- Window (48) - : +- * ColumnarToRow (47) - : +- CometSort (46) - : +- ReusedExchange (45) - +- BroadcastExchange (71) - +- * Project (70) - +- Window (69) - +- * ColumnarToRow (68) - +- CometSort (67) - +- ReusedExchange (66) +TakeOrderedAndProject (85) ++- * Filter (84) + +- * HashAggregate (83) + +- * HashAggregate (82) + +- * Project (81) + +- * BroadcastHashJoin Inner BuildRight (80) + :- Window (73) + : +- * ColumnarToRow (72) + : +- CometSort (71) + : +- CometColumnarExchange (70) + : +- CometProject (69) + : +- CometFilter (68) + : +- CometSortMergeJoin (67) + : :- CometSort (33) + : : +- CometColumnarExchange (32) + : : +- RowToColumnar (31) + : : +- * HashAggregate (30) + : : +- * ColumnarToRow (29) + : : +- CometColumnarExchange (28) + : : +- RowToColumnar (27) + : : +- * HashAggregate (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (17) + : : : +- Window (16) + : : : +- * ColumnarToRow (15) + : : : +- CometSort (14) + : : : +- CometColumnarExchange (13) + : : : +- RowToColumnar (12) + : : : +- * HashAggregate (11) + : : : +- * ColumnarToRow (10) + : : : +- CometColumnarExchange (9) + : : : +- RowToColumnar (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- Window (21) + : : +- * ColumnarToRow (20) + : : +- CometSort (19) + : : +- ReusedExchange (18) + : +- CometSort (66) + : +- CometColumnarExchange (65) + : +- RowToColumnar (64) + : +- * HashAggregate (63) + : +- * ColumnarToRow (62) + : +- CometColumnarExchange (61) + : +- RowToColumnar (60) + : +- * HashAggregate (59) + : +- * Project (58) + : +- * BroadcastHashJoin Inner BuildRight (57) + : :- * Project (50) + : : +- Window (49) + : : +- * ColumnarToRow (48) + : : +- CometSort (47) + : : +- CometColumnarExchange (46) + : : +- RowToColumnar (45) + : : +- * HashAggregate (44) + : : +- * ColumnarToRow (43) + : : +- CometColumnarExchange (42) + : : +- RowToColumnar (41) + : : +- * HashAggregate (40) + : : +- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * ColumnarToRow (36) + : : : +- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.store_sales (34) + : : +- ReusedExchange (37) + : +- BroadcastExchange (56) + : +- * Project (55) + : +- Window (54) + : +- * ColumnarToRow (53) + : +- CometSort (52) + : +- ReusedExchange (51) + +- BroadcastExchange (79) + +- * Project (78) + +- Window (77) + +- * ColumnarToRow (76) + +- CometSort (75) + +- ReusedExchange (74) (unknown) Scan parquet spark_catalog.default.web_sales @@ -93,7 +101,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 2] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 82] +(4) ReusedExchange [Reuses operator id: 90] Output [2]: [d_date_sk#5, d_date#6] (5) BroadcastHashJoin [codegen id : 2] @@ -120,328 +128,360 @@ Input [3]: [ws_item_sk#1, d_date#6, sum#8] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) CometHashAggregate +(10) ColumnarToRow [codegen id : 3] +Input [3]: [ws_item_sk#1, d_date#6, sum#8] + +(11) HashAggregate [codegen id : 3] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] +Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] + +(12) RowToColumnar +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -(11) CometColumnarExchange -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +(13) CometColumnarExchange +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(12) CometSort -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(14) CometSort +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(13) ColumnarToRow [codegen id : 3] -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +(15) ColumnarToRow [codegen id : 4] +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -(14) Window -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(16) Window +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Project [codegen id : 8] -Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] -Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] +(17) Project [codegen id : 10] +Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] +Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] -(16) ReusedExchange [Reuses operator id: 11] -Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +(18) ReusedExchange [Reuses operator id: 13] +Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -(17) CometSort -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] +(19) CometSort +Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +Arguments: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14], [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] -(18) ColumnarToRow [codegen id : 6] -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +(20) ColumnarToRow [codegen id : 8] +Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -(19) Window -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] +(21) Window +Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] -(20) Project [codegen id : 7] -Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] -Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] +(22) Project [codegen id : 9] +Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] +Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] -(21) BroadcastExchange -Input [3]: [item_sk#15, sumws#16, rk#14] +(23) BroadcastExchange +Input [3]: [item_sk#16, sumws#17, rk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [item_sk#9] -Right keys [1]: [item_sk#15] +(24) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [item_sk#10] +Right keys [1]: [item_sk#16] Join type: Inner -Join condition: (rk#11 >= rk#14) +Join condition: (rk#12 >= rk#15) -(23) Project [codegen id : 8] -Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] +(25) Project [codegen id : 10] +Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] +Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] -(24) HashAggregate [codegen id : 8] -Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [partial_sum(sumws#16)] -Aggregate Attributes [2]: [sum#17, isEmpty#18] -Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +(26) HashAggregate [codegen id : 10] +Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] +Keys [3]: [item_sk#10, d_date#6, sumws#11] +Functions [1]: [partial_sum(sumws#17)] +Aggregate Attributes [2]: [sum#18, isEmpty#19] +Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(25) RowToColumnar -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +(27) RowToColumnar +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(26) CometColumnarExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(28) CometColumnarExchange +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(27) CometHashAggregate -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [sum(sumws#16)] +(29) ColumnarToRow [codegen id : 11] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(28) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#21] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(30) HashAggregate [codegen id : 11] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Keys [3]: [item_sk#10, d_date#6, sumws#11] +Functions [1]: [sum(sumws#17)] +Aggregate Attributes [1]: [sum(sumws#17)#22] +Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] + +(31) RowToColumnar +Input [3]: [item_sk#10, d_date#6, cume_sales#23] -(29) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#21] -Arguments: [item_sk#9, d_date#6, cume_sales#21], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(32) CometColumnarExchange +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(33) CometSort +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#22, ss_sales_price#23, ss_sold_date_sk#24] +Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(31) CometFilter -Input [3]: [ss_item_sk#22, ss_sales_price#23, ss_sold_date_sk#24] -Condition : isnotnull(ss_item_sk#22) +(35) CometFilter +Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_item_sk#24) -(32) ColumnarToRow [codegen id : 10] -Input [3]: [ss_item_sk#22, ss_sales_price#23, ss_sold_date_sk#24] +(36) ColumnarToRow [codegen id : 13] +Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -(33) ReusedExchange [Reuses operator id: 82] -Output [2]: [d_date_sk#26, d_date#27] +(37) ReusedExchange [Reuses operator id: 90] +Output [2]: [d_date_sk#28, d_date#29] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#24] -Right keys [1]: [d_date_sk#26] +(38) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_sold_date_sk#26] +Right keys [1]: [d_date_sk#28] Join type: Inner Join condition: None -(35) Project [codegen id : 10] -Output [3]: [ss_item_sk#22, ss_sales_price#23, d_date#27] -Input [5]: [ss_item_sk#22, ss_sales_price#23, ss_sold_date_sk#24, d_date_sk#26, d_date#27] +(39) Project [codegen id : 13] +Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] + +(40) HashAggregate [codegen id : 13] +Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] +Aggregate Attributes [1]: [sum#30] +Results [3]: [ss_item_sk#24, d_date#29, sum#31] + +(41) RowToColumnar +Input [3]: [ss_item_sk#24, d_date#29, sum#31] -(36) HashAggregate [codegen id : 10] -Input [3]: [ss_item_sk#22, ss_sales_price#23, d_date#27] -Keys [2]: [ss_item_sk#22, d_date#27] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#23))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [ss_item_sk#22, d_date#27, sum#29] +(42) CometColumnarExchange +Input [3]: [ss_item_sk#24, d_date#29, sum#31] +Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(37) RowToColumnar -Input [3]: [ss_item_sk#22, d_date#27, sum#29] +(43) ColumnarToRow [codegen id : 14] +Input [3]: [ss_item_sk#24, d_date#29, sum#31] -(38) CometColumnarExchange -Input [3]: [ss_item_sk#22, d_date#27, sum#29] -Arguments: hashpartitioning(ss_item_sk#22, d_date#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(44) HashAggregate [codegen id : 14] +Input [3]: [ss_item_sk#24, d_date#29, sum#31] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#32] +Results [4]: [ss_item_sk#24 AS item_sk#33, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#32,17,2) AS sumss#34, ss_item_sk#24] -(39) CometHashAggregate -Input [3]: [ss_item_sk#22, d_date#27, sum#29] -Keys [2]: [ss_item_sk#22, d_date#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#23))] +(45) RowToColumnar +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -(40) CometColumnarExchange -Input [4]: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22] -Arguments: hashpartitioning(ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(46) CometColumnarExchange +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(41) CometSort -Input [4]: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22] -Arguments: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22], [ss_item_sk#22 ASC NULLS FIRST, d_date#27 ASC NULLS FIRST] +(47) CometSort +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] -(42) ColumnarToRow [codegen id : 11] -Input [4]: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22] +(48) ColumnarToRow [codegen id : 15] +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -(43) Window -Input [4]: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22] -Arguments: [row_number() windowspecdefinition(ss_item_sk#22, d_date#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#22], [d_date#27 ASC NULLS FIRST] +(49) Window +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] -(44) Project [codegen id : 16] -Output [4]: [item_sk#30, d_date#27, sumss#31, rk#32] -Input [5]: [item_sk#30, d_date#27, sumss#31, ss_item_sk#22, rk#32] +(50) Project [codegen id : 21] +Output [4]: [item_sk#33, d_date#29, sumss#34, rk#35] +Input [5]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24, rk#35] -(45) ReusedExchange [Reuses operator id: 40] -Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +(51) ReusedExchange [Reuses operator id: 46] +Output [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -(46) CometSort -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] +(52) CometSort +Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +Arguments: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37], [ss_item_sk#37 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] -(47) ColumnarToRow [codegen id : 14] -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +(53) ColumnarToRow [codegen id : 19] +Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -(48) Window -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] +(54) Window +Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +Arguments: [row_number() windowspecdefinition(ss_item_sk#37, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [ss_item_sk#37], [d_date#36 ASC NULLS FIRST] -(49) Project [codegen id : 15] -Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] -Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] +(55) Project [codegen id : 20] +Output [3]: [item_sk#33 AS item_sk#39, sumss#34 AS sumss#40, rk#38] +Input [5]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37, rk#38] -(50) BroadcastExchange -Input [3]: [item_sk#36, sumss#37, rk#35] +(56) BroadcastExchange +Input [3]: [item_sk#39, sumss#40, rk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [item_sk#30] -Right keys [1]: [item_sk#36] +(57) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [item_sk#33] +Right keys [1]: [item_sk#39] Join type: Inner -Join condition: (rk#32 >= rk#35) +Join condition: (rk#35 >= rk#38) + +(58) Project [codegen id : 21] +Output [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] +Input [7]: [item_sk#33, d_date#29, sumss#34, rk#35, item_sk#39, sumss#40, rk#38] + +(59) HashAggregate [codegen id : 21] +Input [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] +Keys [3]: [item_sk#33, d_date#29, sumss#34] +Functions [1]: [partial_sum(sumss#40)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -(52) Project [codegen id : 16] -Output [4]: [item_sk#30, d_date#27, sumss#31, sumss#37] -Input [7]: [item_sk#30, d_date#27, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] +(60) RowToColumnar +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -(53) HashAggregate [codegen id : 16] -Input [4]: [item_sk#30, d_date#27, sumss#31, sumss#37] -Keys [3]: [item_sk#30, d_date#27, sumss#31] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [item_sk#30, d_date#27, sumss#31, sum#40, isEmpty#41] +(61) CometColumnarExchange +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +Arguments: hashpartitioning(item_sk#33, d_date#29, sumss#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(54) RowToColumnar -Input [5]: [item_sk#30, d_date#27, sumss#31, sum#40, isEmpty#41] +(62) ColumnarToRow [codegen id : 22] +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -(55) CometColumnarExchange -Input [5]: [item_sk#30, d_date#27, sumss#31, sum#40, isEmpty#41] -Arguments: hashpartitioning(item_sk#30, d_date#27, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(63) HashAggregate [codegen id : 22] +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +Keys [3]: [item_sk#33, d_date#29, sumss#34] +Functions [1]: [sum(sumss#40)] +Aggregate Attributes [1]: [sum(sumss#40)#45] +Results [3]: [item_sk#33, d_date#29, sum(sumss#40)#45 AS cume_sales#46] -(56) CometHashAggregate -Input [5]: [item_sk#30, d_date#27, sumss#31, sum#40, isEmpty#41] -Keys [3]: [item_sk#30, d_date#27, sumss#31] -Functions [1]: [sum(sumss#37)] +(64) RowToColumnar +Input [3]: [item_sk#33, d_date#29, cume_sales#46] -(57) CometColumnarExchange -Input [3]: [item_sk#30, d_date#27, cume_sales#42] -Arguments: hashpartitioning(item_sk#30, d_date#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(65) CometColumnarExchange +Input [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: hashpartitioning(item_sk#33, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(58) CometSort -Input [3]: [item_sk#30, d_date#27, cume_sales#42] -Arguments: [item_sk#30, d_date#27, cume_sales#42], [item_sk#30 ASC NULLS FIRST, d_date#27 ASC NULLS FIRST] +(66) CometSort +Input [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: [item_sk#33, d_date#29, cume_sales#46], [item_sk#33 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] -(59) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#21] -Right output [3]: [item_sk#30, d_date#27, cume_sales#42] -Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#27], FullOuter +(67) CometSortMergeJoin +Left output [3]: [item_sk#10, d_date#6, cume_sales#23] +Right output [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: [item_sk#10, d_date#6], [item_sk#33, d_date#29], FullOuter -(60) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#21, item_sk#30, d_date#27, cume_sales#42] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) +(68) CometFilter +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END) -(61) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#21, item_sk#30, d_date#27, cume_sales#42] -Arguments: [item_sk#43, d_date#44, web_sales#45, store_sales#46], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#43, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#27 END AS d_date#44, cume_sales#21 AS web_sales#45, cume_sales#42 AS store_sales#46] +(69) CometProject +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] +Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END AS item_sk#47, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#46 AS store_sales#50] -(62) CometColumnarExchange -Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] -Arguments: hashpartitioning(item_sk#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(70) CometColumnarExchange +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: hashpartitioning(item_sk#47, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(63) CometSort -Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] -Arguments: [item_sk#43, d_date#44, web_sales#45, store_sales#46], [item_sk#43 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST] +(71) CometSort +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] -(64) ColumnarToRow [codegen id : 17] -Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] +(72) ColumnarToRow [codegen id : 23] +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -(65) Window -Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] -Arguments: [row_number() windowspecdefinition(item_sk#43, d_date#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#47], [item_sk#43], [d_date#44 ASC NULLS FIRST] +(73) Window +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#47], [d_date#48 ASC NULLS FIRST] -(66) ReusedExchange [Reuses operator id: 62] -Output [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] +(74) ReusedExchange [Reuses operator id: 70] +Output [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -(67) CometSort -Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] -Arguments: [item_sk#43, d_date#44, web_sales#45, store_sales#46], [item_sk#43 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST] +(75) CometSort +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] -(68) ColumnarToRow [codegen id : 34] -Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] +(76) ColumnarToRow [codegen id : 46] +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -(69) Window -Input [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] -Arguments: [row_number() windowspecdefinition(item_sk#43, d_date#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#43], [d_date#44 ASC NULLS FIRST] +(77) Window +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#52], [item_sk#47], [d_date#48 ASC NULLS FIRST] -(70) Project [codegen id : 35] -Output [4]: [item_sk#43 AS item_sk#49, web_sales#45 AS web_sales#50, store_sales#46 AS store_sales#51, rk#48] -Input [5]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, rk#48] +(78) Project [codegen id : 47] +Output [4]: [item_sk#47 AS item_sk#53, web_sales#49 AS web_sales#54, store_sales#50 AS store_sales#55, rk#52] +Input [5]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#52] -(71) BroadcastExchange -Input [4]: [item_sk#49, web_sales#50, store_sales#51, rk#48] +(79) BroadcastExchange +Input [4]: [item_sk#53, web_sales#54, store_sales#55, rk#52] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(72) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [item_sk#43] -Right keys [1]: [item_sk#49] +(80) BroadcastHashJoin [codegen id : 48] +Left keys [1]: [item_sk#47] +Right keys [1]: [item_sk#53] Join type: Inner -Join condition: (rk#47 >= rk#48) - -(73) Project [codegen id : 36] -Output [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, web_sales#50, store_sales#51] -Input [9]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, rk#47, item_sk#49, web_sales#50, store_sales#51, rk#48] - -(74) HashAggregate [codegen id : 36] -Input [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, web_sales#50, store_sales#51] -Keys [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] -Functions [2]: [partial_max(web_sales#50), partial_max(store_sales#51)] -Aggregate Attributes [2]: [max#52, max#53] -Results [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, max#54, max#55] - -(75) HashAggregate [codegen id : 36] -Input [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, max#54, max#55] -Keys [4]: [item_sk#43, d_date#44, web_sales#45, store_sales#46] -Functions [2]: [max(web_sales#50), max(store_sales#51)] -Aggregate Attributes [2]: [max(web_sales#50)#56, max(store_sales#51)#57] -Results [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, max(web_sales#50)#56 AS web_cumulative#58, max(store_sales#51)#57 AS store_cumulative#59] - -(76) Filter [codegen id : 36] -Input [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, web_cumulative#58, store_cumulative#59] -Condition : ((isnotnull(web_cumulative#58) AND isnotnull(store_cumulative#59)) AND (web_cumulative#58 > store_cumulative#59)) - -(77) TakeOrderedAndProject -Input [6]: [item_sk#43, d_date#44, web_sales#45, store_sales#46, web_cumulative#58, store_cumulative#59] -Arguments: 100, [item_sk#43 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST], [item_sk#43, d_date#44, web_sales#45, store_sales#46, web_cumulative#58, store_cumulative#59] +Join condition: (rk#51 >= rk#52) + +(81) Project [codegen id : 48] +Output [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] +Input [9]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#51, item_sk#53, web_sales#54, store_sales#55, rk#52] + +(82) HashAggregate [codegen id : 48] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] +Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Functions [2]: [partial_max(web_sales#54), partial_max(store_sales#55)] +Aggregate Attributes [2]: [max#56, max#57] +Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] + +(83) HashAggregate [codegen id : 48] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] +Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Functions [2]: [max(web_sales#54), max(store_sales#55)] +Aggregate Attributes [2]: [max(web_sales#54)#60, max(store_sales#55)#61] +Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max(web_sales#54)#60 AS web_cumulative#62, max(store_sales#55)#61 AS store_cumulative#63] + +(84) Filter [codegen id : 48] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] +Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) + +(85) TakeOrderedAndProject +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] +Arguments: 100, [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (82) -+- * ColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (90) ++- * ColumnarToRow (89) + +- CometProject (88) + +- CometFilter (87) + +- CometScan parquet spark_catalog.default.date_dim (86) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#60] +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#64] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(79) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#60] -Condition : (((isnotnull(d_month_seq#60) AND (d_month_seq#60 >= 1212)) AND (d_month_seq#60 <= 1223)) AND isnotnull(d_date_sk#5)) +(87) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) -(80) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#60] +(88) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(81) ColumnarToRow [codegen id : 1] +(89) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(82) BroadcastExchange +(90) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 34 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index ec52058fb9..3fa0c2c698 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (36) + WholeStageCodegen (48) Filter [web_cumulative,store_cumulative] HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] @@ -7,7 +7,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (17) + WholeStageCodegen (23) ColumnarToRow InputAdapter CometSort [item_sk,d_date] @@ -17,102 +17,118 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometSortMergeJoin [item_sk,d_date,item_sk,d_date] CometSort [item_sk,d_date] CometColumnarExchange [item_sk,d_date] #2 - CometHashAggregate [item_sk,d_date,sumws,sum,isEmpty] - CometColumnarExchange [item_sk,d_date,sumws] #3 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (3) - ColumnarToRow + RowToColumnar + WholeStageCodegen (11) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] InputAdapter - CometSort [ws_item_sk,d_date] - CometColumnarExchange [ws_item_sk] #4 - CometHashAggregate [ws_item_sk,d_date,sum] - CometColumnarExchange [ws_item_sk,d_date] #5 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Window [ws_item_sk,d_date] + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometSort [ws_item_sk,d_date] + CometColumnarExchange [ws_item_sk] #4 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow + CometColumnarExchange [ws_item_sk,d_date] #5 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Project [item_sk,sumws,rk] + ReusedExchange [d_date_sk,d_date] #6 InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (6) - ColumnarToRow + BroadcastExchange #7 + WholeStageCodegen (9) + Project [item_sk,sumws,rk] InputAdapter - CometSort [ws_item_sk,d_date] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + Window [ws_item_sk,d_date] + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometSort [ws_item_sk,d_date] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 CometSort [item_sk,d_date] CometColumnarExchange [item_sk,d_date] #8 - CometHashAggregate [item_sk,d_date,sumss,sum,isEmpty] - CometColumnarExchange [item_sk,d_date,sumss] #9 - RowToColumnar - WholeStageCodegen (16) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (11) - ColumnarToRow + RowToColumnar + WholeStageCodegen (22) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #9 + RowToColumnar + WholeStageCodegen (21) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] InputAdapter - CometSort [ss_item_sk,d_date] - CometColumnarExchange [ss_item_sk] #10 - CometHashAggregate [ss_item_sk,d_date,sum] - CometColumnarExchange [ss_item_sk,d_date] #11 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Window [ss_item_sk,d_date] + WholeStageCodegen (15) + ColumnarToRow + InputAdapter + CometSort [ss_item_sk,d_date] + CometColumnarExchange [ss_item_sk] #10 + RowToColumnar + WholeStageCodegen (14) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (15) - Project [item_sk,sumss,rk] + CometColumnarExchange [ss_item_sk,d_date] #11 + RowToColumnar + WholeStageCodegen (13) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (14) - ColumnarToRow + BroadcastExchange #12 + WholeStageCodegen (20) + Project [item_sk,sumss,rk] InputAdapter - CometSort [ss_item_sk,d_date] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + Window [ss_item_sk,d_date] + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometSort [ss_item_sk,d_date] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 InputAdapter BroadcastExchange #13 - WholeStageCodegen (35) + WholeStageCodegen (47) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (34) + WholeStageCodegen (46) ColumnarToRow InputAdapter CometSort [item_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index 966f99500b..f935c102e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -1,51 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * ColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- CometHashAggregate (22) - : : +- CometColumnarExchange (21) - : : +- RowToColumnar (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- Window (34) - : +- * ColumnarToRow (33) - : +- CometSort (32) - : +- ReusedExchange (31) - +- BroadcastExchange (44) - +- * Project (43) - +- Window (42) - +- * ColumnarToRow (41) - +- CometSort (40) - +- ReusedExchange (39) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (32) + : : +- * Filter (31) + : : +- Window (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * ColumnarToRow (27) + : : +- CometSort (26) + : : +- CometColumnarExchange (25) + : : +- RowToColumnar (24) + : : +- * HashAggregate (23) + : : +- * ColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- RowToColumnar (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.call_center (13) + : +- BroadcastExchange (42) + : +- * Project (41) + : +- Window (40) + : +- * ColumnarToRow (39) + : +- CometSort (38) + : +- CometColumnarExchange (37) + : +- RowToColumnar (36) + : +- * HashAggregate (35) + : +- * ColumnarToRow (34) + : +- ReusedExchange (33) + +- BroadcastExchange (50) + +- * Project (49) + +- Window (48) + +- * ColumnarToRow (47) + +- CometSort (46) + +- ReusedExchange (45) (unknown) Scan parquet spark_catalog.default.item @@ -91,7 +97,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 51] +(10) ReusedExchange [Reuses operator id: 57] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -146,117 +152,142 @@ Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometHashAggregate +(22) ColumnarToRow [codegen id : 5] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] + +(23) HashAggregate [codegen id : 5] Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#16] +Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS _w0#18] + +(24) RowToColumnar +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -(23) CometColumnarExchange -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +(25) CometColumnarExchange +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometSort -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] -Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(26) CometSort +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 5] -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +(27) ColumnarToRow [codegen id : 6] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -(26) Window -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#18], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +(28) Window +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(27) Filter [codegen id : 6] -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] +(29) Filter [codegen id : 7] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(28) Window -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] -Arguments: [avg(_w0#17) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#19], [i_category#3, i_brand#2, cc_name#13, d_year#10] +(30) Window +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] + +(31) Filter [codegen id : 22] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) + +(32) Project [codegen id : 22] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] + +(33) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] + +(34) ColumnarToRow [codegen id : 12] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(29) Filter [codegen id : 19] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] -Condition : ((isnotnull(avg_monthly_sales#19) AND (avg_monthly_sales#19 > 0.000000)) AND CASE WHEN (avg_monthly_sales#19 > 0.000000) THEN ((abs((sum_sales#16 - avg_monthly_sales#19)) / avg_monthly_sales#19) > 0.1000000000000000) END) +(35) HashAggregate [codegen id : 12] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] +Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] +Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] -(30) Project [codegen id : 19] -Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] +(36) RowToColumnar +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -(31) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +(37) CometColumnarExchange +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) CometSort -Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] -Arguments: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16], [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, cc_name#22 ASC NULLS FIRST, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] +(38) CometSort +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(33) ColumnarToRow [codegen id : 11] -Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +(39) ColumnarToRow [codegen id : 13] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -(34) Window -Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] -Arguments: [rank(d_year#23, d_moy#24) windowspecdefinition(i_category#20, i_brand#21, cc_name#22, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#25], [i_category#20, i_brand#21, cc_name#22], [d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] +(40) Window +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(35) Project [codegen id : 12] -Output [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#16 AS sum_sales#26, rn#25] -Input [7]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16, rn#25] +(41) Project [codegen id : 14] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] -(36) BroadcastExchange -Input [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#26, rn#25] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=5] +(42) BroadcastExchange +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 19] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] -Right keys [4]: [i_category#20, i_brand#21, cc_name#22, (rn#25 + 1)] +(43) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] +Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(38) Project [codegen id : 19] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#26] -Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, cc_name#22, sum_sales#26, rn#25] +(44) Project [codegen id : 22] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -(39) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] +(45) ReusedExchange [Reuses operator id: 37] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -(40) CometSort -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] -Arguments: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16], [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] +(46) CometSort +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 17] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] +(47) ColumnarToRow [codegen id : 20] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -(42) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16] -Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] +(48) Window +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(43) Project [codegen id : 18] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#16 AS sum_sales#33, rn#32] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#16, rn#32] +(49) Project [codegen id : 21] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] -(44) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#33, rn#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=6] +(50) BroadcastExchange +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(45) BroadcastHashJoin [codegen id : 19] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#32 - 1)] +(51) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] +Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(46) Project [codegen id : 19] -Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#26 AS psum#34, sum_sales#33 AS nsum#35] -Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#26, i_category#27, i_brand#28, cc_name#29, sum_sales#33, rn#32] +(52) Project [codegen id : 22] +Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -(47) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#34, nsum#35] -Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#34, nsum#35] +(53) TakeOrderedAndProject +Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (51) -+- * ColumnarToRow (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (57) ++- * ColumnarToRow (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) (unknown) Scan parquet spark_catalog.default.date_dim @@ -266,15 +297,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter +(55) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(50) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(51) BroadcastExchange +(57) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index 174e9c66ae..a591ad8c2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (19) + WholeStageCodegen (22) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,72 +8,82 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (6) + WholeStageCodegen (7) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy] CometColumnarExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (12) + WholeStageCodegen (14) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (11) + WholeStageCodegen (13) ColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 + CometColumnarExchange [i_category,i_brand,cc_name] #7 + RowToColumnar + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (18) + BroadcastExchange #8 + WholeStageCodegen (21) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (17) + WholeStageCodegen (20) ColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #1 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 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 06fede35c0..d2a6deea76 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,159 +1,167 @@ == Physical Plan == -* ColumnarToRow (155) -+- CometTakeOrderedAndProject (154) - +- CometHashAggregate (153) - +- CometColumnarExchange (152) - +- CometHashAggregate (151) - +- CometUnion (150) - :- CometHashAggregate (75) - : +- CometColumnarExchange (74) - : +- RowToColumnar (73) - : +- * HashAggregate (72) - : +- Union (71) - : :- * HashAggregate (22) - : : +- * ColumnarToRow (21) - : : +- CometColumnarExchange (20) - : : +- RowToColumnar (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (11) - : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : :- * ColumnarToRow (8) - : : : : +- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : +- ReusedExchange (9) - : : +- BroadcastExchange (15) - : : +- * ColumnarToRow (14) - : : +- CometFilter (13) - : : +- CometScan parquet spark_catalog.default.store (12) - : :- * HashAggregate (44) - : : +- * ColumnarToRow (43) - : : +- CometColumnarExchange (42) - : : +- RowToColumnar (41) - : : +- * HashAggregate (40) - : : +- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (33) - : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : :- * ColumnarToRow (30) - : : : : +- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (31) - : : +- BroadcastExchange (37) - : : +- * ColumnarToRow (36) - : : +- CometFilter (35) - : : +- CometScan parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (70) - : +- * ColumnarToRow (69) - : +- CometColumnarExchange (68) - : +- RowToColumnar (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * ColumnarToRow (56) - : : : +- CometUnion (55) - : : : :- CometProject (47) - : : : : +- CometFilter (46) - : : : : +- CometScan parquet spark_catalog.default.web_sales (45) - : : : +- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometBroadcastExchange (49) - : : : : +- CometScan parquet spark_catalog.default.web_returns (48) - : : : +- CometProject (52) - : : : +- CometFilter (51) - : : : +- CometScan parquet spark_catalog.default.web_sales (50) - : : +- ReusedExchange (57) - : +- BroadcastExchange (63) - : +- * ColumnarToRow (62) - : +- CometFilter (61) - : +- CometScan parquet spark_catalog.default.web_site (60) - :- CometHashAggregate (112) - : +- CometColumnarExchange (111) - : +- CometHashAggregate (110) - : +- CometHashAggregate (109) - : +- CometColumnarExchange (108) - : +- RowToColumnar (107) - : +- * HashAggregate (106) - : +- Union (105) - : :- * HashAggregate (78) - : : +- * ColumnarToRow (77) - : : +- ReusedExchange (76) - : :- * HashAggregate (81) - : : +- * ColumnarToRow (80) - : : +- ReusedExchange (79) - : +- * HashAggregate (104) - : +- * ColumnarToRow (103) - : +- CometColumnarExchange (102) - : +- RowToColumnar (101) - : +- * HashAggregate (100) - : +- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- * Project (96) - : : +- * BroadcastHashJoin Inner BuildRight (95) - : : :- * ColumnarToRow (93) - : : : +- CometUnion (92) - : : : :- CometProject (84) - : : : : +- CometFilter (83) - : : : : +- CometScan parquet spark_catalog.default.web_sales (82) - : : : +- CometProject (91) - : : : +- CometBroadcastHashJoin (90) - : : : :- CometBroadcastExchange (86) - : : : : +- CometScan parquet spark_catalog.default.web_returns (85) - : : : +- CometProject (89) - : : : +- CometFilter (88) - : : : +- CometScan parquet spark_catalog.default.web_sales (87) - : : +- ReusedExchange (94) - : +- ReusedExchange (97) - +- CometHashAggregate (149) - +- CometColumnarExchange (148) - +- CometHashAggregate (147) - +- CometHashAggregate (146) - +- CometColumnarExchange (145) - +- RowToColumnar (144) - +- * HashAggregate (143) - +- Union (142) - :- * HashAggregate (115) - : +- * ColumnarToRow (114) - : +- ReusedExchange (113) - :- * HashAggregate (118) - : +- * ColumnarToRow (117) - : +- ReusedExchange (116) - +- * HashAggregate (141) - +- * ColumnarToRow (140) - +- CometColumnarExchange (139) - +- RowToColumnar (138) - +- * HashAggregate (137) - +- * Project (136) - +- * BroadcastHashJoin Inner BuildRight (135) - :- * Project (133) - : +- * BroadcastHashJoin Inner BuildRight (132) - : :- * ColumnarToRow (130) - : : +- CometUnion (129) - : : :- CometProject (121) - : : : +- CometFilter (120) - : : : +- CometScan parquet spark_catalog.default.web_sales (119) - : : +- CometProject (128) - : : +- CometBroadcastHashJoin (127) - : : :- CometBroadcastExchange (123) - : : : +- CometScan parquet spark_catalog.default.web_returns (122) - : : +- CometProject (126) - : : +- CometFilter (125) - : : +- CometScan parquet spark_catalog.default.web_sales (124) - : +- ReusedExchange (131) - +- ReusedExchange (134) +* ColumnarToRow (163) ++- CometTakeOrderedAndProject (162) + +- CometHashAggregate (161) + +- CometColumnarExchange (160) + +- RowToColumnar (159) + +- * HashAggregate (158) + +- Union (157) + :- * HashAggregate (76) + : +- * ColumnarToRow (75) + : +- CometColumnarExchange (74) + : +- RowToColumnar (73) + : +- * HashAggregate (72) + : +- Union (71) + : :- * HashAggregate (22) + : : +- * ColumnarToRow (21) + : : +- CometColumnarExchange (20) + : : +- RowToColumnar (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Project (11) + : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : :- * ColumnarToRow (8) + : : : : +- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : +- ReusedExchange (9) + : : +- BroadcastExchange (15) + : : +- * ColumnarToRow (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.store (12) + : :- * HashAggregate (44) + : : +- * ColumnarToRow (43) + : : +- CometColumnarExchange (42) + : : +- RowToColumnar (41) + : : +- * HashAggregate (40) + : : +- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * Project (33) + : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : :- * ColumnarToRow (30) + : : : : +- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (31) + : : +- BroadcastExchange (37) + : : +- * ColumnarToRow (36) + : : +- CometFilter (35) + : : +- CometScan parquet spark_catalog.default.catalog_page (34) + : +- * HashAggregate (70) + : +- * ColumnarToRow (69) + : +- CometColumnarExchange (68) + : +- RowToColumnar (67) + : +- * HashAggregate (66) + : +- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * ColumnarToRow (56) + : : : +- CometUnion (55) + : : : :- CometProject (47) + : : : : +- CometFilter (46) + : : : : +- CometScan parquet spark_catalog.default.web_sales (45) + : : : +- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometBroadcastExchange (49) + : : : : +- CometScan parquet spark_catalog.default.web_returns (48) + : : : +- CometProject (52) + : : : +- CometFilter (51) + : : : +- CometScan parquet spark_catalog.default.web_sales (50) + : : +- ReusedExchange (57) + : +- BroadcastExchange (63) + : +- * ColumnarToRow (62) + : +- CometFilter (61) + : +- CometScan parquet spark_catalog.default.web_site (60) + :- * HashAggregate (116) + : +- * ColumnarToRow (115) + : +- CometColumnarExchange (114) + : +- RowToColumnar (113) + : +- * HashAggregate (112) + : +- * HashAggregate (111) + : +- * ColumnarToRow (110) + : +- CometColumnarExchange (109) + : +- RowToColumnar (108) + : +- * HashAggregate (107) + : +- Union (106) + : :- * HashAggregate (79) + : : +- * ColumnarToRow (78) + : : +- ReusedExchange (77) + : :- * HashAggregate (82) + : : +- * ColumnarToRow (81) + : : +- ReusedExchange (80) + : +- * HashAggregate (105) + : +- * ColumnarToRow (104) + : +- CometColumnarExchange (103) + : +- RowToColumnar (102) + : +- * HashAggregate (101) + : +- * Project (100) + : +- * BroadcastHashJoin Inner BuildRight (99) + : :- * Project (97) + : : +- * BroadcastHashJoin Inner BuildRight (96) + : : :- * ColumnarToRow (94) + : : : +- CometUnion (93) + : : : :- CometProject (85) + : : : : +- CometFilter (84) + : : : : +- CometScan parquet spark_catalog.default.web_sales (83) + : : : +- CometProject (92) + : : : +- CometBroadcastHashJoin (91) + : : : :- CometBroadcastExchange (87) + : : : : +- CometScan parquet spark_catalog.default.web_returns (86) + : : : +- CometProject (90) + : : : +- CometFilter (89) + : : : +- CometScan parquet spark_catalog.default.web_sales (88) + : : +- ReusedExchange (95) + : +- ReusedExchange (98) + +- * HashAggregate (156) + +- * ColumnarToRow (155) + +- CometColumnarExchange (154) + +- RowToColumnar (153) + +- * HashAggregate (152) + +- * HashAggregate (151) + +- * ColumnarToRow (150) + +- CometColumnarExchange (149) + +- RowToColumnar (148) + +- * HashAggregate (147) + +- Union (146) + :- * HashAggregate (119) + : +- * ColumnarToRow (118) + : +- ReusedExchange (117) + :- * HashAggregate (122) + : +- * ColumnarToRow (121) + : +- ReusedExchange (120) + +- * HashAggregate (145) + +- * ColumnarToRow (144) + +- CometColumnarExchange (143) + +- RowToColumnar (142) + +- * HashAggregate (141) + +- * Project (140) + +- * BroadcastHashJoin Inner BuildRight (139) + :- * Project (137) + : +- * BroadcastHashJoin Inner BuildRight (136) + : :- * ColumnarToRow (134) + : : +- CometUnion (133) + : : :- CometProject (125) + : : : +- CometFilter (124) + : : : +- CometScan parquet spark_catalog.default.web_sales (123) + : : +- CometProject (132) + : : +- CometBroadcastHashJoin (131) + : : :- CometBroadcastExchange (127) + : : : +- CometScan parquet spark_catalog.default.web_returns (126) + : : +- CometProject (130) + : : +- CometFilter (129) + : : +- CometScan parquet spark_catalog.default.web_sales (128) + : +- ReusedExchange (135) + +- ReusedExchange (138) (unknown) Scan parquet spark_catalog.default.store_sales @@ -195,7 +203,7 @@ Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_a (8) ColumnarToRow [codegen id : 3] Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -(9) ReusedExchange [Reuses operator id: 160] +(9) ReusedExchange [Reuses operator id: 168] Output [1]: [d_date_sk#22] (10) BroadcastHashJoin [codegen id : 3] @@ -299,7 +307,7 @@ Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_am (30) ColumnarToRow [codegen id : 7] Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] -(31) ReusedExchange [Reuses operator id: 160] +(31) ReusedExchange [Reuses operator id: 168] Output [1]: [d_date_sk#63] (32) BroadcastHashJoin [codegen id : 7] @@ -422,7 +430,7 @@ Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#10 (56) ColumnarToRow [codegen id : 11] Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(57) ReusedExchange [Reuses operator id: 160] +(57) ReusedExchange [Reuses operator id: 168] Output [1]: [d_date_sk#109] (58) BroadcastHashJoin [codegen id : 11] @@ -503,32 +511,37 @@ Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#1 Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) CometHashAggregate +(75) ColumnarToRow [codegen id : 14] +Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] + +(76) HashAggregate [codegen id : 14] Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] +Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] +Results [5]: [channel#37, id#38, cast(sum(sales#39)#141 as decimal(37,2)) AS sales#144, cast(sum(returns#40)#142 as decimal(37,2)) AS returns#145, cast(sum(profit#41)#143 as decimal(38,2)) AS profit#146] -(76) ReusedExchange [Reuses operator id: 20] -Output [5]: [s_store_id#24, sum#141, sum#142, sum#143, sum#144] +(77) ReusedExchange [Reuses operator id: 20] +Output [5]: [s_store_id#24, sum#147, sum#148, sum#149, sum#150] -(77) ColumnarToRow [codegen id : 17] -Input [5]: [s_store_id#24, sum#141, sum#142, sum#143, sum#144] +(78) ColumnarToRow [codegen id : 18] +Input [5]: [s_store_id#24, sum#147, sum#148, sum#149, sum#150] -(78) HashAggregate [codegen id : 17] -Input [5]: [s_store_id#24, sum#141, sum#142, sum#143, sum#144] +(79) HashAggregate [codegen id : 18] +Input [5]: [s_store_id#24, sum#147, sum#148, sum#149, sum#150] Keys [1]: [s_store_id#24] 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))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] -(79) ReusedExchange [Reuses operator id: 42] -Output [5]: [cp_catalog_page_id#65, sum#145, sum#146, sum#147, sum#148] +(80) ReusedExchange [Reuses operator id: 42] +Output [5]: [cp_catalog_page_id#65, sum#151, sum#152, sum#153, sum#154] -(80) ColumnarToRow [codegen id : 21] -Input [5]: [cp_catalog_page_id#65, sum#145, sum#146, sum#147, sum#148] +(81) ColumnarToRow [codegen id : 22] +Input [5]: [cp_catalog_page_id#65, sum#151, sum#152, sum#153, sum#154] -(81) HashAggregate [codegen id : 21] -Input [5]: [cp_catalog_page_id#65, sum#145, sum#146, sum#147, sum#148] +(82) HashAggregate [codegen id : 22] +Input [5]: [cp_catalog_page_id#65, sum#151, sum#152, sum#153, sum#154] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] @@ -538,15 +551,15 @@ Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_pag Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#149)] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#155)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(83) CometFilter +(84) CometFilter Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Condition : isnotnull(ws_web_site_sk#83) -(84) CometProject +(85) CometProject Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] @@ -554,10 +567,10 @@ Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_am Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#149)] +PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#155)] ReadSchema: struct -(86) CometBroadcastExchange +(87) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] @@ -568,136 +581,151 @@ 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 -(88) CometFilter +(89) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(89) CometProject +(90) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(90) CometBroadcastHashJoin +(91) CometBroadcastHashJoin Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(91) CometProject +(92) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -(92) CometUnion +(93) CometUnion Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] -(93) ColumnarToRow [codegen id : 24] +(94) ColumnarToRow [codegen id : 25] Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(94) ReusedExchange [Reuses operator id: 160] +(95) ReusedExchange [Reuses operator id: 168] Output [1]: [d_date_sk#109] -(95) BroadcastHashJoin [codegen id : 24] +(96) BroadcastHashJoin [codegen id : 25] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(96) Project [codegen id : 24] +(97) Project [codegen id : 25] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] -(97) ReusedExchange [Reuses operator id: 63] +(98) ReusedExchange [Reuses operator id: 63] Output [2]: [web_site_sk#110, web_site_id#111] -(98) BroadcastHashJoin [codegen id : 24] +(99) BroadcastHashJoin [codegen id : 25] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(99) Project [codegen id : 24] +(100) Project [codegen id : 25] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(100) HashAggregate [codegen id : 24] +(101) HashAggregate [codegen id : 25] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#150, sum#151, sum#152, sum#153] -Results [5]: [web_site_id#111, sum#154, sum#155, sum#156, sum#157] +Aggregate Attributes [4]: [sum#156, sum#157, sum#158, sum#159] +Results [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] -(101) RowToColumnar -Input [5]: [web_site_id#111, sum#154, sum#155, sum#156, sum#157] +(102) RowToColumnar +Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] -(102) CometColumnarExchange -Input [5]: [web_site_id#111, sum#154, sum#155, sum#156, sum#157] +(103) CometColumnarExchange +Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(103) ColumnarToRow [codegen id : 25] -Input [5]: [web_site_id#111, sum#154, sum#155, sum#156, sum#157] +(104) ColumnarToRow [codegen id : 26] +Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] -(104) HashAggregate [codegen id : 25] -Input [5]: [web_site_id#111, sum#154, sum#155, sum#156, sum#157] +(105) HashAggregate [codegen id : 26] +Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] -(105) Union +(106) Union -(106) HashAggregate [codegen id : 26] +(107) HashAggregate [codegen id : 27] Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] Keys [2]: [channel#37, id#38] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -(107) RowToColumnar +(108) RowToColumnar Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -(108) CometColumnarExchange +(109) CometColumnarExchange Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(109) CometHashAggregate +(110) ColumnarToRow [codegen id : 28] +Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] + +(111) HashAggregate [codegen id : 28] Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] +Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] +Results [4]: [channel#37, sum(sales#39)#141 AS sales#164, sum(returns#40)#142 AS returns#165, sum(profit#41)#143 AS profit#166] -(110) CometHashAggregate -Input [4]: [channel#37, sales#158, returns#159, profit#160] +(112) HashAggregate [codegen id : 28] +Input [4]: [channel#37, sales#164, returns#165, profit#166] Keys [1]: [channel#37] -Functions [3]: [partial_sum(sales#158), partial_sum(returns#159), partial_sum(profit#160)] +Functions [3]: [partial_sum(sales#164), partial_sum(returns#165), partial_sum(profit#166)] +Aggregate Attributes [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] +Results [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] + +(113) RowToColumnar +Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] -(111) CometColumnarExchange -Input [7]: [channel#37, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166] +(114) CometColumnarExchange +Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(112) CometHashAggregate -Input [7]: [channel#37, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166] +(115) ColumnarToRow [codegen id : 29] +Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] + +(116) HashAggregate [codegen id : 29] +Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Keys [1]: [channel#37] -Functions [3]: [sum(sales#158), sum(returns#159), sum(profit#160)] +Functions [3]: [sum(sales#164), sum(returns#165), sum(profit#166)] +Aggregate Attributes [3]: [sum(sales#164)#179, sum(returns#165)#180, sum(profit#166)#181] +Results [5]: [channel#37, null AS id#182, sum(sales#164)#179 AS sum(sales)#183, sum(returns#165)#180 AS sum(returns)#184, sum(profit#166)#181 AS sum(profit)#185] -(113) ReusedExchange [Reuses operator id: 20] -Output [5]: [s_store_id#24, sum#167, sum#168, sum#169, sum#170] +(117) ReusedExchange [Reuses operator id: 20] +Output [5]: [s_store_id#24, sum#186, sum#187, sum#188, sum#189] -(114) ColumnarToRow [codegen id : 30] -Input [5]: [s_store_id#24, sum#167, sum#168, sum#169, sum#170] +(118) ColumnarToRow [codegen id : 33] +Input [5]: [s_store_id#24, sum#186, sum#187, sum#188, sum#189] -(115) HashAggregate [codegen id : 30] -Input [5]: [s_store_id#24, sum#167, sum#168, sum#169, sum#170] +(119) HashAggregate [codegen id : 33] +Input [5]: [s_store_id#24, sum#186, sum#187, sum#188, sum#189] Keys [1]: [s_store_id#24] 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))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] -(116) ReusedExchange [Reuses operator id: 42] -Output [5]: [cp_catalog_page_id#65, sum#171, sum#172, sum#173, sum#174] +(120) ReusedExchange [Reuses operator id: 42] +Output [5]: [cp_catalog_page_id#65, sum#190, sum#191, sum#192, sum#193] -(117) ColumnarToRow [codegen id : 34] -Input [5]: [cp_catalog_page_id#65, sum#171, sum#172, sum#173, sum#174] +(121) ColumnarToRow [codegen id : 37] +Input [5]: [cp_catalog_page_id#65, sum#190, sum#191, sum#192, sum#193] -(118) HashAggregate [codegen id : 34] -Input [5]: [cp_catalog_page_id#65, sum#171, sum#172, sum#173, sum#174] +(122) HashAggregate [codegen id : 37] +Input [5]: [cp_catalog_page_id#65, sum#190, sum#191, sum#192, sum#193] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] @@ -707,15 +735,15 @@ Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_pag Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#175)] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#194)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(120) CometFilter +(124) CometFilter Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Condition : isnotnull(ws_web_site_sk#83) -(121) CometProject +(125) CometProject Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] @@ -723,10 +751,10 @@ Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_am Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#175)] +PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#194)] ReadSchema: struct -(123) CometBroadcastExchange +(127) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] @@ -737,170 +765,187 @@ 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 -(125) CometFilter +(129) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(126) CometProject +(130) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(127) CometBroadcastHashJoin +(131) CometBroadcastHashJoin Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(128) CometProject +(132) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -(129) CometUnion +(133) CometUnion Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] -(130) ColumnarToRow [codegen id : 37] +(134) ColumnarToRow [codegen id : 40] Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(131) ReusedExchange [Reuses operator id: 160] +(135) ReusedExchange [Reuses operator id: 168] Output [1]: [d_date_sk#109] -(132) BroadcastHashJoin [codegen id : 37] +(136) BroadcastHashJoin [codegen id : 40] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(133) Project [codegen id : 37] +(137) Project [codegen id : 40] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] -(134) ReusedExchange [Reuses operator id: 63] +(138) ReusedExchange [Reuses operator id: 63] Output [2]: [web_site_sk#110, web_site_id#111] -(135) BroadcastHashJoin [codegen id : 37] +(139) BroadcastHashJoin [codegen id : 40] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(136) Project [codegen id : 37] +(140) Project [codegen id : 40] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(137) HashAggregate [codegen id : 37] +(141) HashAggregate [codegen id : 40] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#176, sum#177, sum#178, sum#179] -Results [5]: [web_site_id#111, sum#180, sum#181, sum#182, sum#183] +Aggregate Attributes [4]: [sum#195, sum#196, sum#197, sum#198] +Results [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] -(138) RowToColumnar -Input [5]: [web_site_id#111, sum#180, sum#181, sum#182, sum#183] +(142) RowToColumnar +Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] -(139) CometColumnarExchange -Input [5]: [web_site_id#111, sum#180, sum#181, sum#182, sum#183] +(143) CometColumnarExchange +Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(140) ColumnarToRow [codegen id : 38] -Input [5]: [web_site_id#111, sum#180, sum#181, sum#182, sum#183] +(144) ColumnarToRow [codegen id : 41] +Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] -(141) HashAggregate [codegen id : 38] -Input [5]: [web_site_id#111, sum#180, sum#181, sum#182, sum#183] +(145) HashAggregate [codegen id : 41] +Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] -(142) Union +(146) Union -(143) HashAggregate [codegen id : 39] +(147) HashAggregate [codegen id : 42] Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] Keys [2]: [channel#37, id#38] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -(144) RowToColumnar +(148) RowToColumnar Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -(145) CometColumnarExchange +(149) CometColumnarExchange Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(146) CometHashAggregate +(150) ColumnarToRow [codegen id : 43] +Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] + +(151) HashAggregate [codegen id : 43] Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] +Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] +Results [3]: [sum(sales#39)#141 AS sales#164, sum(returns#40)#142 AS returns#165, sum(profit#41)#143 AS profit#166] -(147) CometHashAggregate -Input [3]: [sales#158, returns#159, profit#160] +(152) HashAggregate [codegen id : 43] +Input [3]: [sales#164, returns#165, profit#166] Keys: [] -Functions [3]: [partial_sum(sales#158), partial_sum(returns#159), partial_sum(profit#160)] +Functions [3]: [partial_sum(sales#164), partial_sum(returns#165), partial_sum(profit#166)] +Aggregate Attributes [6]: [sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208] +Results [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] + +(153) RowToColumnar +Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] -(148) CometColumnarExchange -Input [6]: [sum#184, isEmpty#185, sum#186, isEmpty#187, sum#188, isEmpty#189] +(154) CometColumnarExchange +Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(149) CometHashAggregate -Input [6]: [sum#184, isEmpty#185, sum#186, isEmpty#187, sum#188, isEmpty#189] +(155) ColumnarToRow [codegen id : 44] +Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] + +(156) HashAggregate [codegen id : 44] +Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] Keys: [] -Functions [3]: [sum(sales#158), sum(returns#159), sum(profit#160)] +Functions [3]: [sum(sales#164), sum(returns#165), sum(profit#166)] +Aggregate Attributes [3]: [sum(sales#164)#215, sum(returns#165)#216, sum(profit#166)#217] +Results [5]: [null AS channel#218, null AS id#219, sum(sales#164)#215 AS sum(sales)#220, sum(returns#165)#216 AS sum(returns)#221, sum(profit#166)#217 AS sum(profit)#222] -(150) CometUnion -Child 0 Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] -Child 1 Input [5]: [channel#37, id#193, sum(sales)#194, sum(returns)#195, sum(profit)#196] -Child 2 Input [5]: [channel#197, id#198, sum(sales)#199, sum(returns)#200, sum(profit)#201] +(157) Union -(151) CometHashAggregate -Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] -Keys [5]: [channel#37, id#38, sales#190, returns#191, profit#192] +(158) HashAggregate [codegen id : 45] +Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] + +(159) RowToColumnar +Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -(152) CometColumnarExchange -Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] -Arguments: hashpartitioning(channel#37, id#38, sales#190, returns#191, profit#192, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(160) CometColumnarExchange +Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Arguments: hashpartitioning(channel#37, id#38, sales#144, returns#145, profit#146, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(153) CometHashAggregate -Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] -Keys [5]: [channel#37, id#38, sales#190, returns#191, profit#192] +(161) CometHashAggregate +Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] Functions: [] -(154) CometTakeOrderedAndProject -Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#37 ASC NULLS FIRST,id#38 ASC NULLS FIRST], output=[channel#37,id#38,sales#190,returns#191,profit#192]), 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#190, returns#191, profit#192] +(162) CometTakeOrderedAndProject +Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#37 ASC NULLS FIRST,id#38 ASC NULLS FIRST], output=[channel#37,id#38,sales#144,returns#145,profit#146]), 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#144, returns#145, profit#146] -(155) ColumnarToRow [codegen id : 40] -Input [5]: [channel#37, id#38, sales#190, returns#191, profit#192] +(163) ColumnarToRow [codegen id : 46] +Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (160) -+- * ColumnarToRow (159) - +- CometProject (158) - +- CometFilter (157) - +- CometScan parquet spark_catalog.default.date_dim (156) +BroadcastExchange (168) ++- * ColumnarToRow (167) + +- CometProject (166) + +- CometFilter (165) + +- CometScan parquet spark_catalog.default.date_dim (164) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#202] +Output [2]: [d_date_sk#22, d_date#223] 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 -(157) CometFilter -Input [2]: [d_date_sk#22, d_date#202] -Condition : (((isnotnull(d_date#202) AND (d_date#202 >= 1998-08-04)) AND (d_date#202 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +(165) CometFilter +Input [2]: [d_date_sk#22, d_date#223] +Condition : (((isnotnull(d_date#223) AND (d_date#223 >= 1998-08-04)) AND (d_date#223 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(158) CometProject -Input [2]: [d_date_sk#22, d_date#202] +(166) CometProject +Input [2]: [d_date_sk#22, d_date#223] Arguments: [d_date_sk#22], [d_date_sk#22] -(159) ColumnarToRow [codegen id : 1] +(167) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(160) BroadcastExchange +(168) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] @@ -914,12 +959,12 @@ Subquery:5 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#86 IN d Subquery:6 Hosting operator id = 48 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 -Subquery:7 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:7 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:8 Hosting operator id = 85 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:8 Hosting operator id = 86 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 -Subquery:9 Hosting operator id = 119 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:9 Hosting operator id = 123 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:10 Hosting operator id = 122 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 126 Hosting Expression = wr_returned_date_sk#98 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 d24e3bae4b..ab4910c4a8 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,224 +1,244 @@ -WholeStageCodegen (40) +WholeStageCodegen (46) ColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] CometColumnarExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [channel,id] #2 - RowToColumnar - WholeStageCodegen (13) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (4) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan 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] - CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (8) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #6 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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] - CometScan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (12) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #8 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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,ws_item_sk,ws_order_number] - CometBroadcastExchange #9 - CometScan 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_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - CometHashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [channel] #11 - CometHashAggregate [channel,sales,returns,profit] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [channel,id] #12 - RowToColumnar - WholeStageCodegen (26) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (17) - 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] - ColumnarToRow - InputAdapter - ReusedExchange [s_store_id,sum,sum,sum,sum] #3 - WholeStageCodegen (21) - 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] - ColumnarToRow - InputAdapter - ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 - WholeStageCodegen (25) - 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] + RowToColumnar + WholeStageCodegen (45) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (14) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (13) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan 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] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #6 + RowToColumnar + WholeStageCodegen (7) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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] + CometScan 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 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (12) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #8 + RowToColumnar + WholeStageCodegen (11) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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,ws_item_sk,ws_order_number] + CometBroadcastExchange #9 + CometScan 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_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (29) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #11 + RowToColumnar + WholeStageCodegen (28) + 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] ColumnarToRow InputAdapter - CometColumnarExchange [web_site_id] #13 + CometColumnarExchange [channel,id] #12 RowToColumnar - WholeStageCodegen (24) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] + WholeStageCodegen (27) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (18) + 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] ColumnarToRow InputAdapter - CometUnion - 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] - CometScan 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,ws_item_sk,ws_order_number] - CometBroadcastExchange #14 - CometScan 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_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [web_site_sk,web_site_id] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange #15 - CometHashAggregate [sales,returns,profit] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [channel,id] #16 - RowToColumnar - WholeStageCodegen (39) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (30) - 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] - ColumnarToRow - InputAdapter - ReusedExchange [s_store_id,sum,sum,sum,sum] #3 - WholeStageCodegen (34) - 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] - ColumnarToRow - InputAdapter - ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 - WholeStageCodegen (38) - 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] + ReusedExchange [s_store_id,sum,sum,sum,sum] #3 + WholeStageCodegen (22) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 + WholeStageCodegen (26) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #13 + RowToColumnar + WholeStageCodegen (25) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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,ws_item_sk,ws_order_number] + CometBroadcastExchange #14 + CometScan 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_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [web_site_sk,web_site_id] #10 + WholeStageCodegen (44) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange #15 + RowToColumnar + WholeStageCodegen (43) + 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] ColumnarToRow InputAdapter - CometColumnarExchange [web_site_id] #17 + CometColumnarExchange [channel,id] #16 RowToColumnar - WholeStageCodegen (37) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] + WholeStageCodegen (42) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (33) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [s_store_id,sum,sum,sum,sum] #3 + WholeStageCodegen (37) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 + WholeStageCodegen (41) + 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] ColumnarToRow InputAdapter - CometUnion - 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] - CometScan 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,ws_item_sk,ws_order_number] - CometBroadcastExchange #18 - CometScan 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_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [web_site_sk,web_site_id] #10 + CometColumnarExchange [web_site_id] #17 + RowToColumnar + WholeStageCodegen (40) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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,ws_item_sk,ws_order_number] + CometBroadcastExchange #18 + CometScan 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_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [web_site_sk,web_site_id] #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index ffd243c6af..7c9d8e2e4e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* ColumnarToRow (41) -+- CometTakeOrderedAndProject (40) - +- CometFilter (39) - +- CometHashAggregate (38) +TakeOrderedAndProject (41) ++- * Filter (40) + +- * HashAggregate (39) + +- * ColumnarToRow (38) +- CometColumnarExchange (37) +- RowToColumnar (36) +- * HashAggregate (35) @@ -214,21 +214,23 @@ Input [2]: [ca_state#2, count#19] Input [2]: [ca_state#2, count#19] Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(38) CometHashAggregate +(38) ColumnarToRow [codegen id : 7] +Input [2]: [ca_state#2, count#19] + +(39) HashAggregate [codegen id : 7] Input [2]: [ca_state#2, count#19] Keys [1]: [ca_state#2] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#20] +Results [3]: [ca_state#2 AS state#21, count(1)#20 AS cnt#22, ca_state#2] -(39) CometFilter -Input [3]: [state#20, cnt#21, ca_state#2] -Condition : (cnt#21 >= 10) - -(40) CometTakeOrderedAndProject -Input [3]: [state#20, cnt#21, ca_state#2] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#21 ASC NULLS FIRST,ca_state#2 ASC NULLS FIRST], output=[state#20,cnt#21]), 100, [cnt#21 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#20, cnt#21] +(40) Filter [codegen id : 7] +Input [3]: [state#21, cnt#22, ca_state#2] +Condition : (cnt#22 >= 10) -(41) ColumnarToRow [codegen id : 7] -Input [2]: [state#20, cnt#21] +(41) TakeOrderedAndProject +Input [3]: [state#21, cnt#22, ca_state#2] +Arguments: 100, [cnt#22 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#21, cnt#22] ===== Subqueries ===== @@ -241,18 +243,18 @@ BroadcastExchange (46) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#22] +Output [2]: [d_date_sk#9, d_month_seq#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (43) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#22] -Condition : ((isnotnull(d_month_seq#22) AND (d_month_seq#22 = Subquery scalar-subquery#23, [id=#24])) AND isnotnull(d_date_sk#9)) +Input [2]: [d_date_sk#9, d_month_seq#23] +Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-subquery#24, [id=#25])) AND isnotnull(d_date_sk#9)) (44) CometProject -Input [2]: [d_date_sk#9, d_month_seq#22] +Input [2]: [d_date_sk#9, d_month_seq#23] Arguments: [d_date_sk#9], [d_date_sk#9] (45) ColumnarToRow [codegen id : 1] @@ -262,7 +264,7 @@ Input [1]: [d_date_sk#9] Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#23, [id=#24] +Subquery:2 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#24, [id=#25] * ColumnarToRow (53) +- CometHashAggregate (52) +- CometColumnarExchange (51) @@ -273,35 +275,35 @@ Subquery:2 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquer (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#25, d_year#26, d_moy#27] +Output [3]: [d_month_seq#26, d_year#27, d_moy#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (48) CometFilter -Input [3]: [d_month_seq#25, d_year#26, d_moy#27] -Condition : (((isnotnull(d_year#26) AND isnotnull(d_moy#27)) AND (d_year#26 = 2000)) AND (d_moy#27 = 1)) +Input [3]: [d_month_seq#26, d_year#27, d_moy#28] +Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) (49) CometProject -Input [3]: [d_month_seq#25, d_year#26, d_moy#27] -Arguments: [d_month_seq#25], [d_month_seq#25] +Input [3]: [d_month_seq#26, d_year#27, d_moy#28] +Arguments: [d_month_seq#26], [d_month_seq#26] (50) CometHashAggregate -Input [1]: [d_month_seq#25] -Keys [1]: [d_month_seq#25] +Input [1]: [d_month_seq#26] +Keys [1]: [d_month_seq#26] Functions: [] (51) CometColumnarExchange -Input [1]: [d_month_seq#25] -Arguments: hashpartitioning(d_month_seq#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Input [1]: [d_month_seq#26] +Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] (52) CometHashAggregate -Input [1]: [d_month_seq#25] -Keys [1]: [d_month_seq#25] +Input [1]: [d_month_seq#26] +Keys [1]: [d_month_seq#26] Functions: [] (53) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#25] +Input [1]: [d_month_seq#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index 2efd9c5510..b30a459724 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [cnt,ca_state,state] - CometFilter [cnt] - CometHashAggregate [ca_state,count] +TakeOrderedAndProject [cnt,ca_state,state] + WholeStageCodegen (7) + Filter [cnt] + HashAggregate [ca_state,count] [count(1),state,cnt,count] + ColumnarToRow + InputAdapter CometColumnarExchange [ca_state] #1 RowToColumnar WholeStageCodegen (6) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index 5c22de6679..951fab3e84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -1,73 +1,99 @@ == Physical Plan == -TakeOrderedAndProject (69) -+- * Filter (68) - +- Window (67) - +- * ColumnarToRow (66) - +- CometSort (65) - +- CometColumnarExchange (64) - +- CometUnion (63) - :- CometHashAggregate (22) - : +- CometColumnarExchange (21) - : +- RowToColumnar (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.item (13) - :- CometHashAggregate (27) - : +- CometColumnarExchange (26) - : +- CometHashAggregate (25) - : +- CometHashAggregate (24) - : +- ReusedExchange (23) - :- CometHashAggregate (32) - : +- CometColumnarExchange (31) - : +- CometHashAggregate (30) - : +- CometHashAggregate (29) - : +- ReusedExchange (28) - :- CometHashAggregate (37) - : +- CometColumnarExchange (36) - : +- CometHashAggregate (35) - : +- CometHashAggregate (34) - : +- ReusedExchange (33) - :- CometHashAggregate (42) - : +- CometColumnarExchange (41) - : +- CometHashAggregate (40) - : +- CometHashAggregate (39) - : +- ReusedExchange (38) - :- CometHashAggregate (47) - : +- CometColumnarExchange (46) - : +- CometHashAggregate (45) - : +- CometHashAggregate (44) - : +- ReusedExchange (43) - :- CometHashAggregate (52) - : +- CometColumnarExchange (51) - : +- CometHashAggregate (50) - : +- CometHashAggregate (49) - : +- ReusedExchange (48) - :- CometHashAggregate (57) - : +- CometColumnarExchange (56) - : +- CometHashAggregate (55) - : +- CometHashAggregate (54) - : +- ReusedExchange (53) - +- CometHashAggregate (62) - +- CometColumnarExchange (61) - +- CometHashAggregate (60) - +- CometHashAggregate (59) - +- ReusedExchange (58) +TakeOrderedAndProject (95) ++- * Filter (94) + +- Window (93) + +- * ColumnarToRow (92) + +- CometSort (91) + +- CometColumnarExchange (90) + +- RowToColumnar (89) + +- Union (88) + :- * HashAggregate (23) + : +- * ColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- RowToColumnar (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.item (13) + :- * HashAggregate (31) + : +- * ColumnarToRow (30) + : +- CometColumnarExchange (29) + : +- RowToColumnar (28) + : +- * HashAggregate (27) + : +- * HashAggregate (26) + : +- * ColumnarToRow (25) + : +- ReusedExchange (24) + :- * HashAggregate (39) + : +- * ColumnarToRow (38) + : +- CometColumnarExchange (37) + : +- RowToColumnar (36) + : +- * HashAggregate (35) + : +- * HashAggregate (34) + : +- * ColumnarToRow (33) + : +- ReusedExchange (32) + :- * HashAggregate (47) + : +- * ColumnarToRow (46) + : +- CometColumnarExchange (45) + : +- RowToColumnar (44) + : +- * HashAggregate (43) + : +- * HashAggregate (42) + : +- * ColumnarToRow (41) + : +- ReusedExchange (40) + :- * HashAggregate (55) + : +- * ColumnarToRow (54) + : +- CometColumnarExchange (53) + : +- RowToColumnar (52) + : +- * HashAggregate (51) + : +- * HashAggregate (50) + : +- * ColumnarToRow (49) + : +- ReusedExchange (48) + :- * HashAggregate (63) + : +- * ColumnarToRow (62) + : +- CometColumnarExchange (61) + : +- RowToColumnar (60) + : +- * HashAggregate (59) + : +- * HashAggregate (58) + : +- * ColumnarToRow (57) + : +- ReusedExchange (56) + :- * HashAggregate (71) + : +- * ColumnarToRow (70) + : +- CometColumnarExchange (69) + : +- RowToColumnar (68) + : +- * HashAggregate (67) + : +- * HashAggregate (66) + : +- * ColumnarToRow (65) + : +- ReusedExchange (64) + :- * HashAggregate (79) + : +- * ColumnarToRow (78) + : +- CometColumnarExchange (77) + : +- RowToColumnar (76) + : +- * HashAggregate (75) + : +- * HashAggregate (74) + : +- * ColumnarToRow (73) + : +- ReusedExchange (72) + +- * HashAggregate (87) + +- * ColumnarToRow (86) + +- CometColumnarExchange (85) + +- RowToColumnar (84) + +- * HashAggregate (83) + +- * HashAggregate (82) + +- * ColumnarToRow (81) + +- ReusedExchange (80) (unknown) Scan parquet spark_catalog.default.store_sales @@ -85,7 +111,7 @@ Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 74] +(4) ReusedExchange [Reuses operator id: 100] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (5) BroadcastHashJoin [codegen id : 4] @@ -168,250 +194,369 @@ Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometHashAggregate +(22) ColumnarToRow [codegen id : 5] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] + +(23) HashAggregate [codegen id : 5] Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] +Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23] + +(24) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25] -(23) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#22, isEmpty#23] +(25) ColumnarToRow [codegen id : 10] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25] -(24) CometHashAggregate -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#22, isEmpty#23] +(26) HashAggregate [codegen id : 10] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] +Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(25) CometHashAggregate -Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sumsales#24] +(27) HashAggregate [codegen id : 10] +Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sumsales#26] Keys [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9] -Functions [1]: [partial_sum(sumsales#24)] +Functions [1]: [partial_sum(sumsales#26)] +Aggregate Attributes [2]: [sum#27, isEmpty#28] +Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] -(26) CometColumnarExchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#25, isEmpty#26] +(28) RowToColumnar +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] + +(29) CometColumnarExchange +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(27) CometHashAggregate -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#25, isEmpty#26] +(30) ColumnarToRow [codegen id : 11] +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] + +(31) HashAggregate [codegen id : 11] +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] Keys [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9] -Functions [1]: [sum(sumsales#24)] +Functions [1]: [sum(sumsales#26)] +Aggregate Attributes [1]: [sum(sumsales#26)#31] +Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, null AS s_store_id#32, sum(sumsales#26)#31 AS sumsales#33] -(28) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#27, isEmpty#28] +(32) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35] -(29) CometHashAggregate -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#27, isEmpty#28] +(33) ColumnarToRow [codegen id : 16] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35] + +(34) HashAggregate [codegen id : 16] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] +Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(30) CometHashAggregate -Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sumsales#24] +(35) HashAggregate [codegen id : 16] +Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sumsales#26] Keys [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10] -Functions [1]: [partial_sum(sumsales#24)] +Functions [1]: [partial_sum(sumsales#26)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] + +(36) RowToColumnar +Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] -(31) CometColumnarExchange -Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#29, isEmpty#30] +(37) CometColumnarExchange +Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) CometHashAggregate -Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#29, isEmpty#30] +(38) ColumnarToRow [codegen id : 17] +Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] + +(39) HashAggregate [codegen id : 17] +Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] Keys [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10] -Functions [1]: [sum(sumsales#24)] +Functions [1]: [sum(sumsales#26)] +Aggregate Attributes [1]: [sum(sumsales#26)#40] +Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, null AS d_moy#41, null AS s_store_id#42, sum(sumsales#26)#40 AS sumsales#43] + +(40) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45] -(33) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#31, isEmpty#32] +(41) ColumnarToRow [codegen id : 22] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45] -(34) CometHashAggregate -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#31, isEmpty#32] +(42) HashAggregate [codegen id : 22] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] +Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(35) CometHashAggregate -Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sumsales#24] +(43) HashAggregate [codegen id : 22] +Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sumsales#26] Keys [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8] -Functions [1]: [partial_sum(sumsales#24)] +Functions [1]: [partial_sum(sumsales#26)] +Aggregate Attributes [2]: [sum#46, isEmpty#47] +Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] -(36) CometColumnarExchange -Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#33, isEmpty#34] +(44) RowToColumnar +Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] + +(45) CometColumnarExchange +Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(37) CometHashAggregate -Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#33, isEmpty#34] +(46) ColumnarToRow [codegen id : 23] +Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] + +(47) HashAggregate [codegen id : 23] +Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] Keys [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8] -Functions [1]: [sum(sumsales#24)] +Functions [1]: [sum(sumsales#26)] +Aggregate Attributes [1]: [sum(sumsales#26)#50] +Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, null AS d_qoy#51, null AS d_moy#52, null AS s_store_id#53, sum(sumsales#26)#50 AS sumsales#54] + +(48) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56] -(38) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#35, isEmpty#36] +(49) ColumnarToRow [codegen id : 28] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56] -(39) CometHashAggregate -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#35, isEmpty#36] +(50) HashAggregate [codegen id : 28] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] +Results [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(40) CometHashAggregate -Input [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sumsales#24] +(51) HashAggregate [codegen id : 28] +Input [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sumsales#26] Keys [4]: [i_category#16, i_class#15, i_brand#14, i_product_name#17] -Functions [1]: [partial_sum(sumsales#24)] +Functions [1]: [partial_sum(sumsales#26)] +Aggregate Attributes [2]: [sum#57, isEmpty#58] +Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] -(41) CometColumnarExchange -Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#37, isEmpty#38] +(52) RowToColumnar +Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] + +(53) CometColumnarExchange +Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(42) CometHashAggregate -Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#37, isEmpty#38] +(54) ColumnarToRow [codegen id : 29] +Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] + +(55) HashAggregate [codegen id : 29] +Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] Keys [4]: [i_category#16, i_class#15, i_brand#14, i_product_name#17] -Functions [1]: [sum(sumsales#24)] +Functions [1]: [sum(sumsales#26)] +Aggregate Attributes [1]: [sum(sumsales#26)#61] +Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, null AS d_year#62, null AS d_qoy#63, null AS d_moy#64, null AS s_store_id#65, sum(sumsales#26)#61 AS sumsales#66] -(43) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#39, isEmpty#40] +(56) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68] -(44) CometHashAggregate -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#39, isEmpty#40] +(57) ColumnarToRow [codegen id : 34] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68] + +(58) HashAggregate [codegen id : 34] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] +Results [4]: [i_category#16, i_class#15, i_brand#14, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(45) CometHashAggregate -Input [4]: [i_category#16, i_class#15, i_brand#14, sumsales#24] +(59) HashAggregate [codegen id : 34] +Input [4]: [i_category#16, i_class#15, i_brand#14, sumsales#26] Keys [3]: [i_category#16, i_class#15, i_brand#14] -Functions [1]: [partial_sum(sumsales#24)] +Functions [1]: [partial_sum(sumsales#26)] +Aggregate Attributes [2]: [sum#69, isEmpty#70] +Results [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] + +(60) RowToColumnar +Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] -(46) CometColumnarExchange -Input [5]: [i_category#16, i_class#15, i_brand#14, sum#41, isEmpty#42] +(61) CometColumnarExchange +Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(47) CometHashAggregate -Input [5]: [i_category#16, i_class#15, i_brand#14, sum#41, isEmpty#42] +(62) ColumnarToRow [codegen id : 35] +Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] + +(63) HashAggregate [codegen id : 35] +Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] Keys [3]: [i_category#16, i_class#15, i_brand#14] -Functions [1]: [sum(sumsales#24)] +Functions [1]: [sum(sumsales#26)] +Aggregate Attributes [1]: [sum(sumsales#26)#73] +Results [9]: [i_category#16, i_class#15, i_brand#14, null AS i_product_name#74, null AS d_year#75, null AS d_qoy#76, null AS d_moy#77, null AS s_store_id#78, sum(sumsales#26)#73 AS sumsales#79] -(48) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#43, isEmpty#44] +(64) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81] + +(65) ColumnarToRow [codegen id : 40] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81] -(49) CometHashAggregate -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#43, isEmpty#44] +(66) HashAggregate [codegen id : 40] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] +Results [3]: [i_category#16, i_class#15, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(50) CometHashAggregate -Input [3]: [i_category#16, i_class#15, sumsales#24] +(67) HashAggregate [codegen id : 40] +Input [3]: [i_category#16, i_class#15, sumsales#26] Keys [2]: [i_category#16, i_class#15] -Functions [1]: [partial_sum(sumsales#24)] +Functions [1]: [partial_sum(sumsales#26)] +Aggregate Attributes [2]: [sum#82, isEmpty#83] +Results [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] -(51) CometColumnarExchange -Input [4]: [i_category#16, i_class#15, sum#45, isEmpty#46] +(68) RowToColumnar +Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] + +(69) CometColumnarExchange +Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] Arguments: hashpartitioning(i_category#16, i_class#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(52) CometHashAggregate -Input [4]: [i_category#16, i_class#15, sum#45, isEmpty#46] +(70) ColumnarToRow [codegen id : 41] +Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] + +(71) HashAggregate [codegen id : 41] +Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] Keys [2]: [i_category#16, i_class#15] -Functions [1]: [sum(sumsales#24)] +Functions [1]: [sum(sumsales#26)] +Aggregate Attributes [1]: [sum(sumsales#26)#86] +Results [9]: [i_category#16, i_class#15, null AS i_brand#87, null AS i_product_name#88, null AS d_year#89, null AS d_qoy#90, null AS d_moy#91, null AS s_store_id#92, sum(sumsales#26)#86 AS sumsales#93] -(53) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#47, isEmpty#48] +(72) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95] -(54) CometHashAggregate -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#47, isEmpty#48] +(73) ColumnarToRow [codegen id : 46] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95] + +(74) HashAggregate [codegen id : 46] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] +Results [2]: [i_category#16, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(55) CometHashAggregate -Input [2]: [i_category#16, sumsales#24] +(75) HashAggregate [codegen id : 46] +Input [2]: [i_category#16, sumsales#26] Keys [1]: [i_category#16] -Functions [1]: [partial_sum(sumsales#24)] +Functions [1]: [partial_sum(sumsales#26)] +Aggregate Attributes [2]: [sum#96, isEmpty#97] +Results [3]: [i_category#16, sum#98, isEmpty#99] + +(76) RowToColumnar +Input [3]: [i_category#16, sum#98, isEmpty#99] -(56) CometColumnarExchange -Input [3]: [i_category#16, sum#49, isEmpty#50] +(77) CometColumnarExchange +Input [3]: [i_category#16, sum#98, isEmpty#99] Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(57) CometHashAggregate -Input [3]: [i_category#16, sum#49, isEmpty#50] +(78) ColumnarToRow [codegen id : 47] +Input [3]: [i_category#16, sum#98, isEmpty#99] + +(79) HashAggregate [codegen id : 47] +Input [3]: [i_category#16, sum#98, isEmpty#99] Keys [1]: [i_category#16] -Functions [1]: [sum(sumsales#24)] +Functions [1]: [sum(sumsales#26)] +Aggregate Attributes [1]: [sum(sumsales#26)#100] +Results [9]: [i_category#16, null AS i_class#101, null AS i_brand#102, null AS i_product_name#103, null AS d_year#104, null AS d_qoy#105, null AS d_moy#106, null AS s_store_id#107, sum(sumsales#26)#100 AS sumsales#108] + +(80) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110] -(58) ReusedExchange [Reuses operator id: 21] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#51, isEmpty#52] +(81) ColumnarToRow [codegen id : 52] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110] -(59) CometHashAggregate -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#51, isEmpty#52] +(82) HashAggregate [codegen id : 52] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] +Results [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(60) CometHashAggregate -Input [1]: [sumsales#24] +(83) HashAggregate [codegen id : 52] +Input [1]: [sumsales#26] Keys: [] -Functions [1]: [partial_sum(sumsales#24)] +Functions [1]: [partial_sum(sumsales#26)] +Aggregate Attributes [2]: [sum#111, isEmpty#112] +Results [2]: [sum#113, isEmpty#114] -(61) CometColumnarExchange -Input [2]: [sum#53, isEmpty#54] +(84) RowToColumnar +Input [2]: [sum#113, isEmpty#114] + +(85) CometColumnarExchange +Input [2]: [sum#113, isEmpty#114] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(62) CometHashAggregate -Input [2]: [sum#53, isEmpty#54] +(86) ColumnarToRow [codegen id : 53] +Input [2]: [sum#113, isEmpty#114] + +(87) HashAggregate [codegen id : 53] +Input [2]: [sum#113, isEmpty#114] Keys: [] -Functions [1]: [sum(sumsales#24)] - -(63) CometUnion -Child 0 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55] -Child 1 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#56, sumsales#57] -Child 2 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#58, s_store_id#59, sumsales#60] -Child 3 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#61, d_moy#62, s_store_id#63, sumsales#64] -Child 4 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#65, d_qoy#66, d_moy#67, s_store_id#68, sumsales#69] -Child 5 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#70, d_year#71, d_qoy#72, d_moy#73, s_store_id#74, sumsales#75] -Child 6 Input [9]: [i_category#16, i_class#15, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sumsales#82] -Child 7 Input [9]: [i_category#16, i_class#83, i_brand#84, i_product_name#85, d_year#86, d_qoy#87, d_moy#88, s_store_id#89, sumsales#90] -Child 8 Input [9]: [i_category#91, i_class#92, i_brand#93, i_product_name#94, d_year#95, d_qoy#96, d_moy#97, s_store_id#98, sumsales#99] - -(64) CometColumnarExchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55] +Functions [1]: [sum(sumsales#26)] +Aggregate Attributes [1]: [sum(sumsales#26)#115] +Results [9]: [null AS i_category#116, null AS i_class#117, null AS i_brand#118, null AS i_product_name#119, null AS d_year#120, null AS d_qoy#121, null AS d_moy#122, null AS s_store_id#123, sum(sumsales#26)#115 AS sumsales#124] + +(88) Union + +(89) RowToColumnar +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] + +(90) CometColumnarExchange +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(65) CometSort -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55] -Arguments: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55], [i_category#16 ASC NULLS FIRST, sumsales#55 DESC NULLS LAST] +(91) CometSort +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] +Arguments: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23], [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST] -(66) ColumnarToRow [codegen id : 37] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55] +(92) ColumnarToRow [codegen id : 54] +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -(67) Window -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55] -Arguments: [rank(sumsales#55) windowspecdefinition(i_category#16, sumsales#55 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#100], [i_category#16], [sumsales#55 DESC NULLS LAST] +(93) Window +Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] +Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#125], [i_category#16], [sumsales#23 DESC NULLS LAST] -(68) Filter [codegen id : 38] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55, rk#100] -Condition : (rk#100 <= 100) +(94) Filter [codegen id : 55] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] +Condition : (rk#125 <= 100) -(69) TakeOrderedAndProject -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55, rk#100] -Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#55 ASC NULLS FIRST, rk#100 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#55, rk#100] +(95) TakeOrderedAndProject +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] +Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#125 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (74) -+- * ColumnarToRow (73) - +- CometProject (72) - +- CometFilter (71) - +- CometScan parquet spark_catalog.default.date_dim (70) +BroadcastExchange (100) ++- * ColumnarToRow (99) + +- CometProject (98) + +- CometFilter (97) + +- CometScan parquet spark_catalog.default.date_dim (96) (unknown) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#101, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(71) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#101, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#101) AND (d_month_seq#101 >= 1212)) AND (d_month_seq#101 <= 1223)) AND isnotnull(d_date_sk#7)) +(97) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#126) AND (d_month_seq#126 >= 1212)) AND (d_month_seq#126 <= 1223)) AND isnotnull(d_date_sk#7)) -(72) CometProject -Input [5]: [d_date_sk#7, d_month_seq#101, d_year#8, d_moy#9, d_qoy#10] +(98) CometProject +Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(73) ColumnarToRow [codegen id : 1] +(99) ColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(74) BroadcastExchange +(100) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index 1949bc1306..72f33d82e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -1,90 +1,150 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (38) + WholeStageCodegen (55) Filter [rk] InputAdapter Window [sumsales,i_category] - WholeStageCodegen (37) + WholeStageCodegen (54) ColumnarToRow InputAdapter CometSort [i_category,sumsales] CometColumnarExchange [i_category] #1 - CometUnion - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow + RowToColumnar + Union + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + WholeStageCodegen (11) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (17) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 + RowToColumnar + WholeStageCodegen (16) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (23) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 + RowToColumnar + WholeStageCodegen (22) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (29) + HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 + RowToColumnar + WholeStageCodegen (28) + HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (35) + HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand] #10 + RowToColumnar + WholeStageCodegen (34) + HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (41) + HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #11 + RowToColumnar + WholeStageCodegen (40) + HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (47) + HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #12 + RowToColumnar + WholeStageCodegen (46) + HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (53) + HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange #13 + RowToColumnar + WholeStageCodegen (52) + HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 - CometHashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [i_category,i_class,i_brand,sum,isEmpty] - CometColumnarExchange [i_category,i_class,i_brand] #10 - CometHashAggregate [i_category,i_class,i_brand,sumsales] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [i_category,i_class,sum,isEmpty] - CometColumnarExchange [i_category,i_class] #11 - CometHashAggregate [i_category,i_class,sumsales] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [i_category,sum,isEmpty] - CometColumnarExchange [i_category] #12 - CometHashAggregate [i_category,sumsales] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] - CometColumnarExchange #13 - CometHashAggregate [sumsales] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index c3d9525d52..ffae6c6604 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -1,63 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (59) -+- * Project (58) - +- Window (57) - +- * ColumnarToRow (56) - +- CometSort (55) - +- CometColumnarExchange (54) - +- CometHashAggregate (53) - +- CometColumnarExchange (52) - +- CometHashAggregate (51) - +- CometUnion (50) - :- CometHashAggregate (39) - : +- CometColumnarExchange (38) - : +- RowToColumnar (37) - : +- * HashAggregate (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (33) - : +- * BroadcastHashJoin LeftSemi BuildRight (32) - : :- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- Window (28) - : +- * ColumnarToRow (27) - : +- CometSort (26) - : +- CometHashAggregate (25) - : +- CometColumnarExchange (24) - : +- RowToColumnar (23) - : +- * HashAggregate (22) - : +- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * ColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.store_sales (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- ReusedExchange (19) - :- CometHashAggregate (44) - : +- CometColumnarExchange (43) - : +- CometHashAggregate (42) - : +- CometHashAggregate (41) - : +- ReusedExchange (40) - +- CometHashAggregate (49) - +- CometColumnarExchange (48) - +- CometHashAggregate (47) - +- CometHashAggregate (46) - +- ReusedExchange (45) +TakeOrderedAndProject (67) ++- * Project (66) + +- Window (65) + +- * ColumnarToRow (64) + +- CometSort (63) + +- CometColumnarExchange (62) + +- CometHashAggregate (61) + +- CometColumnarExchange (60) + +- RowToColumnar (59) + +- * HashAggregate (58) + +- Union (57) + :- * HashAggregate (40) + : +- * ColumnarToRow (39) + : +- CometColumnarExchange (38) + : +- RowToColumnar (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (33) + : +- * BroadcastHashJoin LeftSemi BuildRight (32) + : :- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- Window (28) + : +- * Sort (27) + : +- * HashAggregate (26) + : +- * ColumnarToRow (25) + : +- CometColumnarExchange (24) + : +- RowToColumnar (23) + : +- * HashAggregate (22) + : +- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * ColumnarToRow (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.store_sales (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- ReusedExchange (19) + :- * HashAggregate (48) + : +- * ColumnarToRow (47) + : +- CometColumnarExchange (46) + : +- RowToColumnar (45) + : +- * HashAggregate (44) + : +- * HashAggregate (43) + : +- * ColumnarToRow (42) + : +- ReusedExchange (41) + +- * HashAggregate (56) + +- * ColumnarToRow (55) + +- CometColumnarExchange (54) + +- RowToColumnar (53) + +- * HashAggregate (52) + +- * HashAggregate (51) + +- * ColumnarToRow (50) + +- ReusedExchange (49) (unknown) Scan parquet spark_catalog.default.store_sales @@ -75,7 +83,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 8] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 64] +(4) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 8] @@ -145,7 +153,7 @@ Join condition: None Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] -(19) ReusedExchange [Reuses operator id: 64] +(19) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#15] (20) BroadcastHashJoin [codegen id : 4] @@ -172,29 +180,31 @@ Input [2]: [s_state#14, sum#17] Input [2]: [s_state#14, sum#17] Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(25) CometHashAggregate +(25) ColumnarToRow [codegen id : 5] +Input [2]: [s_state#14, sum#17] + +(26) HashAggregate [codegen id : 5] Input [2]: [s_state#14, sum#17] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] +Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] -(26) CometSort -Input [3]: [s_state#14, _w0#18, s_state#14] -Arguments: [s_state#14, _w0#18, s_state#14], [s_state#14 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] - -(27) ColumnarToRow [codegen id : 5] -Input [3]: [s_state#14, _w0#18, s_state#14] +(27) Sort [codegen id : 5] +Input [3]: [s_state#14, _w0#19, s_state#14] +Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 (28) Window -Input [3]: [s_state#14, _w0#18, s_state#14] -Arguments: [rank(_w0#18) windowspecdefinition(s_state#14, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#14], [_w0#18 DESC NULLS LAST] +Input [3]: [s_state#14, _w0#19, s_state#14] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] (29) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] -Condition : (ranking#19 <= 5) +Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] +Condition : (ranking#20 <= 5) (30) Project [codegen id : 6] Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] +Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] (31) BroadcastExchange Input [1]: [s_state#14] @@ -224,136 +234,173 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] Input [3]: [ss_net_profit#2, s_county#7, s_state#8] Keys [2]: [s_state#8, s_county#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#20] -Results [3]: [s_state#8, s_county#7, sum#21] +Aggregate Attributes [1]: [sum#21] +Results [3]: [s_state#8, s_county#7, sum#22] (37) RowToColumnar -Input [3]: [s_state#8, s_county#7, sum#21] +Input [3]: [s_state#8, s_county#7, sum#22] (38) CometColumnarExchange -Input [3]: [s_state#8, s_county#7, sum#21] +Input [3]: [s_state#8, s_county#7, sum#22] Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(39) CometHashAggregate -Input [3]: [s_state#8, s_county#7, sum#21] +(39) ColumnarToRow [codegen id : 9] +Input [3]: [s_state#8, s_county#7, sum#22] + +(40) HashAggregate [codegen id : 9] +Input [3]: [s_state#8, s_county#7, sum#22] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) as decimal(27,2)) AS total_sum#24, s_state#8, s_county#7, 0 AS g_state#25, 0 AS g_county#26, 0 AS lochierarchy#27] -(40) ReusedExchange [Reuses operator id: 38] -Output [3]: [s_state#8, s_county#7, sum#22] +(41) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_state#8, s_county#7, sum#28] -(41) CometHashAggregate -Input [3]: [s_state#8, s_county#7, sum#22] +(42) ColumnarToRow [codegen id : 18] +Input [3]: [s_state#8, s_county#7, sum#28] + +(43) HashAggregate [codegen id : 18] +Input [3]: [s_state#8, s_county#7, sum#28] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS total_sum#29, s_state#8] -(42) CometHashAggregate -Input [2]: [total_sum#23, s_state#8] +(44) HashAggregate [codegen id : 18] +Input [2]: [total_sum#29, s_state#8] Keys [1]: [s_state#8] -Functions [1]: [partial_sum(total_sum#23)] +Functions [1]: [partial_sum(total_sum#29)] +Aggregate Attributes [2]: [sum#30, isEmpty#31] +Results [3]: [s_state#8, sum#32, isEmpty#33] + +(45) RowToColumnar +Input [3]: [s_state#8, sum#32, isEmpty#33] -(43) CometColumnarExchange -Input [3]: [s_state#8, sum#24, isEmpty#25] +(46) CometColumnarExchange +Input [3]: [s_state#8, sum#32, isEmpty#33] Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(44) CometHashAggregate -Input [3]: [s_state#8, sum#24, isEmpty#25] +(47) ColumnarToRow [codegen id : 19] +Input [3]: [s_state#8, sum#32, isEmpty#33] + +(48) HashAggregate [codegen id : 19] +Input [3]: [s_state#8, sum#32, isEmpty#33] Keys [1]: [s_state#8] -Functions [1]: [sum(total_sum#23)] +Functions [1]: [sum(total_sum#29)] +Aggregate Attributes [1]: [sum(total_sum#29)#34] +Results [6]: [sum(total_sum#29)#34 AS total_sum#35, s_state#8, null AS s_county#36, 0 AS g_state#37, 1 AS g_county#38, 1 AS lochierarchy#39] -(45) ReusedExchange [Reuses operator id: 38] -Output [3]: [s_state#8, s_county#7, sum#26] +(49) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_state#8, s_county#7, sum#40] -(46) CometHashAggregate -Input [3]: [s_state#8, s_county#7, sum#26] +(50) ColumnarToRow [codegen id : 28] +Input [3]: [s_state#8, s_county#7, sum#40] + +(51) HashAggregate [codegen id : 28] +Input [3]: [s_state#8, s_county#7, sum#40] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS total_sum#29] -(47) CometHashAggregate -Input [1]: [total_sum#23] +(52) HashAggregate [codegen id : 28] +Input [1]: [total_sum#29] Keys: [] -Functions [1]: [partial_sum(total_sum#23)] +Functions [1]: [partial_sum(total_sum#29)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [2]: [sum#43, isEmpty#44] + +(53) RowToColumnar +Input [2]: [sum#43, isEmpty#44] -(48) CometColumnarExchange -Input [2]: [sum#27, isEmpty#28] +(54) CometColumnarExchange +Input [2]: [sum#43, isEmpty#44] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(49) CometHashAggregate -Input [2]: [sum#27, isEmpty#28] +(55) ColumnarToRow [codegen id : 29] +Input [2]: [sum#43, isEmpty#44] + +(56) HashAggregate [codegen id : 29] +Input [2]: [sum#43, isEmpty#44] Keys: [] -Functions [1]: [sum(total_sum#23)] +Functions [1]: [sum(total_sum#29)] +Aggregate Attributes [1]: [sum(total_sum#29)#45] +Results [6]: [sum(total_sum#29)#45 AS total_sum#46, null AS s_state#47, null AS s_county#48, 1 AS g_state#49, 1 AS g_county#50, 2 AS lochierarchy#51] -(50) CometUnion -Child 0 Input [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] -Child 1 Input [6]: [total_sum#33, s_state#8, s_county#34, g_state#35, g_county#36, lochierarchy#37] -Child 2 Input [6]: [total_sum#38, s_state#39, s_county#40, g_state#41, g_county#42, lochierarchy#43] +(57) Union -(51) CometHashAggregate -Input [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] -Keys [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] +(58) HashAggregate [codegen id : 30] +Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] Functions: [] +Aggregate Attributes: [] +Results [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -(52) CometColumnarExchange -Input [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] -Arguments: hashpartitioning(total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +(59) RowToColumnar +Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -(53) CometHashAggregate -Input [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] -Keys [6]: [total_sum#29, s_state#8, s_county#7, g_state#30, g_county#31, lochierarchy#32] +(60) CometColumnarExchange +Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +Arguments: hashpartitioning(total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(61) CometHashAggregate +Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] Functions: [] -(54) CometColumnarExchange -Input [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44] -Arguments: hashpartitioning(lochierarchy#32, _w0#44, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(62) CometColumnarExchange +Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] +Arguments: hashpartitioning(lochierarchy#27, _w0#52, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(55) CometSort -Input [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44] -Arguments: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44], [lochierarchy#32 ASC NULLS FIRST, _w0#44 ASC NULLS FIRST, total_sum#29 DESC NULLS LAST] +(63) CometSort +Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] +Arguments: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52], [lochierarchy#27 ASC NULLS FIRST, _w0#52 ASC NULLS FIRST, total_sum#24 DESC NULLS LAST] -(56) ColumnarToRow [codegen id : 25] -Input [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44] +(64) ColumnarToRow [codegen id : 31] +Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] -(57) Window -Input [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44] -Arguments: [rank(total_sum#29) windowspecdefinition(lochierarchy#32, _w0#44, total_sum#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#45], [lochierarchy#32, _w0#44], [total_sum#29 DESC NULLS LAST] +(65) Window +Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] +Arguments: [rank(total_sum#24) windowspecdefinition(lochierarchy#27, _w0#52, total_sum#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#53], [lochierarchy#27, _w0#52], [total_sum#24 DESC NULLS LAST] -(58) Project [codegen id : 26] -Output [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, rank_within_parent#45] -Input [6]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, _w0#44, rank_within_parent#45] +(66) Project [codegen id : 32] +Output [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] +Input [6]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52, rank_within_parent#53] -(59) TakeOrderedAndProject -Input [5]: [total_sum#29, s_state#8, s_county#7, lochierarchy#32, rank_within_parent#45] -Arguments: 100, [lochierarchy#32 DESC NULLS LAST, CASE WHEN (lochierarchy#32 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#45 ASC NULLS FIRST], [total_sum#29, s_state#8, s_county#7, lochierarchy#32, rank_within_parent#45] +(67) TakeOrderedAndProject +Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#53 ASC NULLS FIRST], [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (64) -+- * ColumnarToRow (63) - +- CometProject (62) - +- CometFilter (61) - +- CometScan parquet spark_catalog.default.date_dim (60) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#46] +Output [2]: [d_date_sk#5, d_month_seq#54] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(61) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#46] -Condition : (((isnotnull(d_month_seq#46) AND (d_month_seq#46 >= 1212)) AND (d_month_seq#46 <= 1223)) AND isnotnull(d_date_sk#5)) +(69) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#54] +Condition : (((isnotnull(d_month_seq#54) AND (d_month_seq#54 >= 1212)) AND (d_month_seq#54 <= 1223)) AND isnotnull(d_date_sk#5)) -(62) CometProject -Input [2]: [d_date_sk#5, d_month_seq#46] +(70) CometProject +Input [2]: [d_date_sk#5, d_month_seq#54] Arguments: [d_date_sk#5], [d_date_sk#5] -(63) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(64) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 4719e8ea07..1a6112bed1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -1,89 +1,109 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (26) + WholeStageCodegen (32) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (25) + WholeStageCodegen (31) ColumnarToRow InputAdapter CometSort [lochierarchy,_w0,total_sum] CometColumnarExchange [lochierarchy,_w0] #1 CometHashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - CometHashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - CometUnion - CometHashAggregate [s_state,s_county,sum] - CometColumnarExchange [s_state,s_county] #3 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - BroadcastHashJoin [s_state,s_state] - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] + RowToColumnar + WholeStageCodegen (30) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (9) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - Window [_w0,s_state] - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [s_state,sum] - CometColumnarExchange [s_state] #7 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - ColumnarToRow + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + BroadcastHashJoin [s_state,s_state] + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WholeStageCodegen (5) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #7 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #4 - CometHashAggregate [s_state,sum,isEmpty] - CometColumnarExchange [s_state] #9 - CometHashAggregate [s_state,total_sum] - CometHashAggregate [s_state,s_county,sum] - ReusedExchange [s_state,s_county,sum] #3 - CometHashAggregate [sum,isEmpty] - CometColumnarExchange #10 - CometHashAggregate [total_sum] - CometHashAggregate [s_state,s_county,sum] - ReusedExchange [s_state,s_county,sum] #3 + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (19) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #9 + RowToColumnar + WholeStageCodegen (18) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + ColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (29) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (28) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + ColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index 46315a3c4f..bb7f608784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -5,9 +5,9 @@ TakeOrderedAndProject (79) :- * Project (58) : +- * BroadcastHashJoin Inner BuildRight (57) : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometHashAggregate (16) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * ColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- RowToColumnar (14) : : : +- * HashAggregate (13) @@ -24,8 +24,8 @@ TakeOrderedAndProject (79) : : : : +- CometScan parquet spark_catalog.default.store_sales (4) : : : +- ReusedExchange (10) : : +- BroadcastExchange (36) - : : +- * ColumnarToRow (35) - : : +- CometHashAggregate (34) + : : +- * HashAggregate (35) + : : +- * ColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- RowToColumnar (32) : : +- * HashAggregate (31) @@ -42,9 +42,9 @@ TakeOrderedAndProject (79) : : : +- CometScan parquet spark_catalog.default.store_sales (22) : : +- ReusedExchange (28) : +- BroadcastExchange (56) - : +- * ColumnarToRow (55) - : +- CometFilter (54) - : +- CometHashAggregate (53) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- * ColumnarToRow (53) : +- CometColumnarExchange (52) : +- RowToColumnar (51) : +- * HashAggregate (50) @@ -61,8 +61,8 @@ TakeOrderedAndProject (79) : : +- CometScan parquet spark_catalog.default.web_sales (41) : +- ReusedExchange (47) +- BroadcastExchange (76) - +- * ColumnarToRow (75) - +- CometHashAggregate (74) + +- * HashAggregate (75) + +- * ColumnarToRow (74) +- CometColumnarExchange (73) +- RowToColumnar (72) +- * HashAggregate (71) @@ -150,297 +150,305 @@ Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 16] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] + +(17) HashAggregate [codegen id : 16] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] +Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] -(17) CometFilter -Input [2]: [customer_id#13, year_total#14] -Condition : (isnotnull(year_total#14) AND (year_total#14 > 0.00)) - -(18) ColumnarToRow [codegen id : 16] -Input [2]: [customer_id#13, year_total#14] +(18) Filter [codegen id : 16] +Input [2]: [customer_id#14, year_total#15] +Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] +Output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (20) CometFilter -Input [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_customer_id#16)) +Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] +Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] (unknown) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (23) CometFilter -Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_customer_sk#19) +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#20) (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] (25) BroadcastExchange -Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#15] -Right keys [1]: [ss_customer_sk#19] +Left keys [1]: [c_customer_sk#16] +Right keys [1]: [ss_customer_sk#20] Join type: Inner Join condition: None (27) Project [codegen id : 6] -Output [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21] -Input [7]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18, ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] +Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] (28) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#23, d_year#24] +Output [2]: [d_date_sk#24, d_year#25] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#21] -Right keys [1]: [d_date_sk#23] +Left keys [1]: [ss_sold_date_sk#22] +Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None (30) Project [codegen id : 6] -Output [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] -Input [7]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21, d_date_sk#23, d_year#24] +Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] +Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] (31) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] -Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#20))] -Aggregate Attributes [1]: [sum#25] -Results [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] +Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] +Aggregate Attributes [1]: [sum#26] +Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] (32) RowToColumnar -Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] (33) CometColumnarExchange -Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] -Arguments: hashpartitioning(c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] +Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometHashAggregate -Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#26] -Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] -Functions [1]: [sum(UnscaledValue(ss_net_paid#20))] +(34) ColumnarToRow [codegen id : 7] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -(35) ColumnarToRow [codegen id : 7] -Input [4]: [customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30] +(35) HashAggregate [codegen id : 7] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] +Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] +Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] +Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] (36) BroadcastExchange -Input [4]: [customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30] +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] (37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#13] -Right keys [1]: [customer_id#27] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#28] Join type: Inner Join condition: None (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] +Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (39) CometFilter -Input [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] -Condition : (isnotnull(c_customer_sk#31) AND isnotnull(c_customer_id#32)) +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) (40) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34] +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] +Output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (42) CometFilter -Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] -Condition : isnotnull(ws_bill_customer_sk#35) +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_bill_customer_sk#36) (43) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] (44) BroadcastExchange -Input [3]: [ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] (45) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#31] -Right keys [1]: [ws_bill_customer_sk#35] +Left keys [1]: [c_customer_sk#32] +Right keys [1]: [ws_bill_customer_sk#36] Join type: Inner Join condition: None (46) Project [codegen id : 10] -Output [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, ws_sold_date_sk#37] -Input [7]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, ws_bill_customer_sk#35, ws_net_paid#36, ws_sold_date_sk#37] +Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] +Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] (47) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#39, d_year#40] +Output [2]: [d_date_sk#40, d_year#41] (48) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None (49) Project [codegen id : 10] -Output [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, d_year#40] -Input [7]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, ws_sold_date_sk#37, d_date_sk#39, d_year#40] +Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] +Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] (50) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, ws_net_paid#36, d_year#40] -Keys [4]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#36))] -Aggregate Attributes [1]: [sum#41] -Results [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] +Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] +Aggregate Attributes [1]: [sum#42] +Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] (51) RowToColumnar -Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] (52) CometColumnarExchange -Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] -Arguments: hashpartitioning(c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometHashAggregate -Input [5]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40, sum#42] -Keys [4]: [c_customer_id#32, c_first_name#33, c_last_name#34, d_year#40] -Functions [1]: [sum(UnscaledValue(ws_net_paid#36))] +(53) ColumnarToRow [codegen id : 11] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -(54) CometFilter -Input [2]: [customer_id#43, year_total#44] -Condition : (isnotnull(year_total#44) AND (year_total#44 > 0.00)) +(54) HashAggregate [codegen id : 11] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] +Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] +Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] +Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] -(55) ColumnarToRow [codegen id : 11] -Input [2]: [customer_id#43, year_total#44] +(55) Filter [codegen id : 11] +Input [2]: [customer_id#45, year_total#46] +Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) (56) BroadcastExchange -Input [2]: [customer_id#43, year_total#44] +Input [2]: [customer_id#45, year_total#46] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] (57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#13] -Right keys [1]: [customer_id#43] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#45] Join type: Inner Join condition: None (58) Project [codegen id : 16] -Output [7]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, year_total#44] -Input [8]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, customer_id#43, year_total#44] +Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] +Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] (unknown) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] +Output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (60) CometFilter -Input [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] -Condition : (isnotnull(c_customer_sk#45) AND isnotnull(c_customer_id#46)) +Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] +Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] (unknown) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] +Output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (63) CometFilter -Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_bill_customer_sk#49) +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_bill_customer_sk#51) (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] (65) BroadcastExchange -Input [3]: [ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] (66) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#45] -Right keys [1]: [ws_bill_customer_sk#49] +Left keys [1]: [c_customer_sk#47] +Right keys [1]: [ws_bill_customer_sk#51] Join type: Inner Join condition: None (67) Project [codegen id : 14] -Output [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, ws_sold_date_sk#51] -Input [7]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48, ws_bill_customer_sk#49, ws_net_paid#50, ws_sold_date_sk#51] +Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] +Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] (68) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#53, d_year#54] +Output [2]: [d_date_sk#55, d_year#56] (69) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#53] +Left keys [1]: [ws_sold_date_sk#53] +Right keys [1]: [d_date_sk#55] Join type: Inner Join condition: None (70) Project [codegen id : 14] -Output [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, d_year#54] -Input [7]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, ws_sold_date_sk#51, d_date_sk#53, d_year#54] +Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] +Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] (71) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#50, d_year#54] -Keys [4]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#50))] -Aggregate Attributes [1]: [sum#55] -Results [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] +Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] +Aggregate Attributes [1]: [sum#57] +Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] (72) RowToColumnar -Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] (73) CometColumnarExchange -Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] -Arguments: hashpartitioning(c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometHashAggregate -Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54, sum#56] -Keys [4]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#54] -Functions [1]: [sum(UnscaledValue(ws_net_paid#50))] +(74) ColumnarToRow [codegen id : 15] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -(75) ColumnarToRow [codegen id : 15] -Input [2]: [customer_id#57, year_total#58] +(75) HashAggregate [codegen id : 15] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] +Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#52))#44] +Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#52))#44,17,2) AS year_total#60] (76) BroadcastExchange -Input [2]: [customer_id#57, year_total#58] +Input [2]: [customer_id#59, year_total#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] (77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#13] -Right keys [1]: [customer_id#57] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#59] Join type: Inner -Join condition: (CASE WHEN (year_total#44 > 0.00) THEN (year_total#58 / year_total#44) END > CASE WHEN (year_total#14 > 0.00) THEN (year_total#30 / year_total#14) END) +Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) (78) Project [codegen id : 16] -Output [3]: [customer_id#27, customer_first_name#28, customer_last_name#29] -Input [9]: [customer_id#13, year_total#14, customer_id#27, customer_first_name#28, customer_last_name#29, year_total#30, year_total#44, customer_id#57, year_total#58] +Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] (79) TakeOrderedAndProject -Input [3]: [customer_id#27, customer_first_name#28, customer_last_name#29] -Arguments: 100, [customer_first_name#28 ASC NULLS FIRST, customer_id#27 ASC NULLS FIRST, customer_last_name#29 ASC NULLS FIRST], [customer_id#27, customer_first_name#28, customer_last_name#29] +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== @@ -469,7 +477,7 @@ Input [2]: [d_date_sk#9, d_year#10] Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (87) +- * ColumnarToRow (86) +- CometFilter (85) @@ -477,25 +485,25 @@ BroadcastExchange (87) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_year#24] +Output [2]: [d_date_sk#24, d_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (85) CometFilter -Input [2]: [d_date_sk#23, d_year#24] -Condition : (((isnotnull(d_year#24) AND (d_year#24 = 2002)) AND d_year#24 IN (2001,2002)) AND isnotnull(d_date_sk#23)) +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) (86) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#23, d_year#24] +Input [2]: [d_date_sk#24, d_year#25] (87) BroadcastExchange -Input [2]: [d_date_sk#23, d_year#24] +Input [2]: [d_date_sk#24, d_year#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#22 +Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index e8b5c93db2..aa010c7b45 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -5,10 +5,10 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] - ColumnarToRow - InputAdapter - CometFilter [year_total] - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 RowToColumnar WholeStageCodegen (3) @@ -40,9 +40,9 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 RowToColumnar WholeStageCodegen (6) @@ -74,10 +74,10 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometFilter [year_total] - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 RowToColumnar WholeStageCodegen (10) @@ -103,9 +103,9 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + ColumnarToRow + InputAdapter CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 RowToColumnar WholeStageCodegen (14) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt index 688a02fb0e..b2b0068947 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt @@ -1,116 +1,124 @@ == Physical Plan == -* ColumnarToRow (112) -+- CometTakeOrderedAndProject (111) - +- CometHashAggregate (110) - +- CometColumnarExchange (109) - +- CometHashAggregate (108) - +- CometUnion (107) - :- CometHashAggregate (96) - : +- CometColumnarExchange (95) - : +- RowToColumnar (94) - : +- * HashAggregate (93) - : +- Union (92) - : :- * Project (34) - : : +- * BroadcastHashJoin LeftOuter BuildRight (33) - : : :- * ColumnarToRow (17) - : : : +- CometHashAggregate (16) - : : : +- CometColumnarExchange (15) - : : : +- RowToColumnar (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.store (7) - : : +- BroadcastExchange (32) - : : +- * ColumnarToRow (31) - : : +- CometHashAggregate (30) - : : +- CometColumnarExchange (29) - : : +- RowToColumnar (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * ColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometScan parquet spark_catalog.default.store_returns (18) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : :- * Project (57) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (56) - : : :- BroadcastExchange (45) - : : : +- * ColumnarToRow (44) - : : : +- CometHashAggregate (43) - : : : +- CometColumnarExchange (42) - : : : +- RowToColumnar (41) - : : : +- * HashAggregate (40) - : : : +- * Project (39) - : : : +- * BroadcastHashJoin Inner BuildRight (38) - : : : :- * ColumnarToRow (36) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (35) - : : : +- ReusedExchange (37) - : : +- * ColumnarToRow (55) - : : +- CometHashAggregate (54) - : : +- CometColumnarExchange (53) - : : +- RowToColumnar (52) - : : +- * HashAggregate (51) - : : +- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * ColumnarToRow (47) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (46) - : : +- ReusedExchange (48) - : +- * Project (91) - : +- * BroadcastHashJoin LeftOuter BuildRight (90) - : :- * ColumnarToRow (74) - : : +- CometHashAggregate (73) - : : +- CometColumnarExchange (72) - : : +- RowToColumnar (71) - : : +- * HashAggregate (70) - : : +- * Project (69) - : : +- * BroadcastHashJoin Inner BuildRight (68) - : : :- * Project (63) - : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : :- * ColumnarToRow (60) - : : : : +- CometFilter (59) - : : : : +- CometScan parquet spark_catalog.default.web_sales (58) - : : : +- ReusedExchange (61) - : : +- BroadcastExchange (67) - : : +- * ColumnarToRow (66) - : : +- CometFilter (65) - : : +- CometScan parquet spark_catalog.default.web_page (64) - : +- BroadcastExchange (89) - : +- * ColumnarToRow (88) - : +- CometHashAggregate (87) - : +- CometColumnarExchange (86) - : +- RowToColumnar (85) - : +- * HashAggregate (84) - : +- * Project (83) - : +- * BroadcastHashJoin Inner BuildRight (82) - : :- * Project (80) - : : +- * BroadcastHashJoin Inner BuildRight (79) - : : :- * ColumnarToRow (77) - : : : +- CometFilter (76) - : : : +- CometScan parquet spark_catalog.default.web_returns (75) - : : +- ReusedExchange (78) - : +- ReusedExchange (81) - :- CometHashAggregate (101) - : +- CometColumnarExchange (100) - : +- CometHashAggregate (99) - : +- CometHashAggregate (98) - : +- ReusedExchange (97) - +- CometHashAggregate (106) - +- CometColumnarExchange (105) - +- CometHashAggregate (104) - +- CometHashAggregate (103) - +- ReusedExchange (102) +* ColumnarToRow (120) ++- CometTakeOrderedAndProject (119) + +- CometHashAggregate (118) + +- CometColumnarExchange (117) + +- RowToColumnar (116) + +- * HashAggregate (115) + +- Union (114) + :- * HashAggregate (97) + : +- * ColumnarToRow (96) + : +- CometColumnarExchange (95) + : +- RowToColumnar (94) + : +- * HashAggregate (93) + : +- Union (92) + : :- * Project (34) + : : +- * BroadcastHashJoin LeftOuter BuildRight (33) + : : :- * HashAggregate (17) + : : : +- * ColumnarToRow (16) + : : : +- CometColumnarExchange (15) + : : : +- RowToColumnar (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan parquet spark_catalog.default.store (7) + : : +- BroadcastExchange (32) + : : +- * HashAggregate (31) + : : +- * ColumnarToRow (30) + : : +- CometColumnarExchange (29) + : : +- RowToColumnar (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * ColumnarToRow (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.store_returns (18) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : :- * Project (57) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (56) + : : :- BroadcastExchange (45) + : : : +- * HashAggregate (44) + : : : +- * ColumnarToRow (43) + : : : +- CometColumnarExchange (42) + : : : +- RowToColumnar (41) + : : : +- * HashAggregate (40) + : : : +- * Project (39) + : : : +- * BroadcastHashJoin Inner BuildRight (38) + : : : :- * ColumnarToRow (36) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (35) + : : : +- ReusedExchange (37) + : : +- * HashAggregate (55) + : : +- * ColumnarToRow (54) + : : +- CometColumnarExchange (53) + : : +- RowToColumnar (52) + : : +- * HashAggregate (51) + : : +- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * ColumnarToRow (47) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (46) + : : +- ReusedExchange (48) + : +- * Project (91) + : +- * BroadcastHashJoin LeftOuter BuildRight (90) + : :- * HashAggregate (74) + : : +- * ColumnarToRow (73) + : : +- CometColumnarExchange (72) + : : +- RowToColumnar (71) + : : +- * HashAggregate (70) + : : +- * Project (69) + : : +- * BroadcastHashJoin Inner BuildRight (68) + : : :- * Project (63) + : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : :- * ColumnarToRow (60) + : : : : +- CometFilter (59) + : : : : +- CometScan parquet spark_catalog.default.web_sales (58) + : : : +- ReusedExchange (61) + : : +- BroadcastExchange (67) + : : +- * ColumnarToRow (66) + : : +- CometFilter (65) + : : +- CometScan parquet spark_catalog.default.web_page (64) + : +- BroadcastExchange (89) + : +- * HashAggregate (88) + : +- * ColumnarToRow (87) + : +- CometColumnarExchange (86) + : +- RowToColumnar (85) + : +- * HashAggregate (84) + : +- * Project (83) + : +- * BroadcastHashJoin Inner BuildRight (82) + : :- * Project (80) + : : +- * BroadcastHashJoin Inner BuildRight (79) + : : :- * ColumnarToRow (77) + : : : +- CometFilter (76) + : : : +- CometScan parquet spark_catalog.default.web_returns (75) + : : +- ReusedExchange (78) + : +- ReusedExchange (81) + :- * HashAggregate (105) + : +- * ColumnarToRow (104) + : +- CometColumnarExchange (103) + : +- RowToColumnar (102) + : +- * HashAggregate (101) + : +- * HashAggregate (100) + : +- * ColumnarToRow (99) + : +- ReusedExchange (98) + +- * HashAggregate (113) + +- * ColumnarToRow (112) + +- CometColumnarExchange (111) + +- RowToColumnar (110) + +- * HashAggregate (109) + +- * HashAggregate (108) + +- * ColumnarToRow (107) + +- ReusedExchange (106) (unknown) Scan parquet spark_catalog.default.store_sales @@ -128,7 +136,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 3] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 117] +(4) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -183,479 +191,528 @@ Input [3]: [s_store_sk#7, sum#10, sum#11] Input [3]: [s_store_sk#7, sum#10, sum#11] Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 8] +Input [3]: [s_store_sk#7, sum#10, sum#11] + +(17) HashAggregate [codegen id : 8] Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) ColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sales#12, profit#13] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] +Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] (unknown) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] +Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(sr_returned_date_sk#17 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (19) CometFilter -Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] -Condition : isnotnull(sr_store_sk#14) +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#16) (20) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -(21) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#19] +(21) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#21] (22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#17] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [d_date_sk#21] Join type: Inner Join condition: None (23) Project [codegen id : 6] -Output [3]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16] -Input [5]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, sr_returned_date_sk#17, d_date_sk#19] +Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] +Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#20] +Output [1]: [s_store_sk#22] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#14] -Right keys [1]: [s_store_sk#20] +Left keys [1]: [sr_store_sk#16] +Right keys [1]: [s_store_sk#22] Join type: Inner Join condition: None (26) Project [codegen id : 6] -Output [3]: [sr_return_amt#15, sr_net_loss#16, s_store_sk#20] -Input [4]: [sr_store_sk#14, sr_return_amt#15, sr_net_loss#16, s_store_sk#20] +Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] (27) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#15, sr_net_loss#16, s_store_sk#20] -Keys [1]: [s_store_sk#20] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#15)), partial_sum(UnscaledValue(sr_net_loss#16))] -Aggregate Attributes [2]: [sum#21, sum#22] -Results [3]: [s_store_sk#20, sum#23, sum#24] +Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Keys [1]: [s_store_sk#22] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum#23, sum#24] +Results [3]: [s_store_sk#22, sum#25, sum#26] (28) RowToColumnar -Input [3]: [s_store_sk#20, sum#23, sum#24] +Input [3]: [s_store_sk#22, sum#25, sum#26] (29) CometColumnarExchange -Input [3]: [s_store_sk#20, sum#23, sum#24] -Arguments: hashpartitioning(s_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Input [3]: [s_store_sk#22, sum#25, sum#26] +Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometHashAggregate -Input [3]: [s_store_sk#20, sum#23, sum#24] -Keys [1]: [s_store_sk#20] -Functions [2]: [sum(UnscaledValue(sr_return_amt#15)), sum(UnscaledValue(sr_net_loss#16))] +(30) ColumnarToRow [codegen id : 7] +Input [3]: [s_store_sk#22, sum#25, sum#26] -(31) ColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#20, returns#25, profit_loss#26] +(31) HashAggregate [codegen id : 7] +Input [3]: [s_store_sk#22, sum#25, sum#26] +Keys [1]: [s_store_sk#22] +Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] +Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] (32) BroadcastExchange -Input [3]: [s_store_sk#20, returns#25, profit_loss#26] +Input [3]: [s_store_sk#22, returns#29, profit_loss#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#20] +Right keys [1]: [s_store_sk#22] Join type: LeftOuter Join condition: None (34) Project [codegen id : 8] -Output [5]: [store channel AS channel#27, s_store_sk#7 AS id#28, sales#12, coalesce(returns#25, 0.00) AS returns#29, (profit#13 - coalesce(profit_loss#26, 0.00)) AS profit#30] -Input [6]: [s_store_sk#7, sales#12, profit#13, s_store_sk#20, returns#25, profit_loss#26] +Output [5]: [store channel AS channel#31, s_store_sk#7 AS id#32, sales#14, coalesce(returns#29, 0.00) AS returns#33, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#34] +Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] -(37) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#36] +(37) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#40] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#34] -Right keys [1]: [d_date_sk#36] +Left keys [1]: [cs_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33] -Input [5]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, d_date_sk#36] +Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#31, cs_ext_sales_price#32, cs_net_profit#33] -Keys [1]: [cs_call_center_sk#31] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(UnscaledValue(cs_net_profit#33))] -Aggregate Attributes [2]: [sum#37, sum#38] -Results [3]: [cs_call_center_sk#31, sum#39, sum#40] +Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum#41, sum#42] +Results [3]: [cs_call_center_sk#35, sum#43, sum#44] (41) RowToColumnar -Input [3]: [cs_call_center_sk#31, sum#39, sum#40] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] (42) CometColumnarExchange -Input [3]: [cs_call_center_sk#31, sum#39, sum#40] -Arguments: hashpartitioning(cs_call_center_sk#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) CometHashAggregate -Input [3]: [cs_call_center_sk#31, sum#39, sum#40] -Keys [1]: [cs_call_center_sk#31] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(UnscaledValue(cs_net_profit#33))] +(43) ColumnarToRow [codegen id : 11] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -(44) ColumnarToRow [codegen id : 11] -Input [3]: [cs_call_center_sk#31, sales#41, profit#42] +(44) HashAggregate [codegen id : 11] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] +Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] (45) BroadcastExchange -Input [3]: [cs_call_center_sk#31, sales#41, profit#42] +Input [3]: [cs_call_center_sk#35, sales#47, profit#48] Arguments: IdentityBroadcastMode, [plan_id=6] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] ReadSchema: struct (47) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -(48) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#47] +(48) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#53] (49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#45] -Right keys [1]: [d_date_sk#47] +Left keys [1]: [cr_returned_date_sk#51] +Right keys [1]: [d_date_sk#53] Join type: Inner Join condition: None (50) Project [codegen id : 13] -Output [2]: [cr_return_amount#43, cr_net_loss#44] -Input [4]: [cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45, d_date_sk#47] +Output [2]: [cr_return_amount#49, cr_net_loss#50] +Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] (51) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#43, cr_net_loss#44] +Input [2]: [cr_return_amount#49, cr_net_loss#50] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#43)), partial_sum(UnscaledValue(cr_net_loss#44))] -Aggregate Attributes [2]: [sum#48, sum#49] -Results [2]: [sum#50, sum#51] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum#54, sum#55] +Results [2]: [sum#56, sum#57] (52) RowToColumnar -Input [2]: [sum#50, sum#51] +Input [2]: [sum#56, sum#57] (53) CometColumnarExchange -Input [2]: [sum#50, sum#51] +Input [2]: [sum#56, sum#57] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(54) CometHashAggregate -Input [2]: [sum#50, sum#51] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#43)), sum(UnscaledValue(cr_net_loss#44))] +(54) ColumnarToRow +Input [2]: [sum#56, sum#57] -(55) ColumnarToRow -Input [2]: [returns#52, profit_loss#53] +(55) HashAggregate +Input [2]: [sum#56, sum#57] +Keys: [] +Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] (56) BroadcastNestedLoopJoin [codegen id : 14] Join type: Inner Join condition: None (57) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#54, cs_call_center_sk#31 AS id#55, sales#41, returns#52, (profit#42 - profit_loss#53) AS profit#56] -Input [5]: [cs_call_center_sk#31, sales#41, profit#42, returns#52, profit_loss#53] +Output [5]: [catalog channel AS channel#62, cs_call_center_sk#35 AS id#63, sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#64] +Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] (unknown) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] +Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (59) CometFilter -Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] -Condition : isnotnull(ws_web_page_sk#57) +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +Condition : isnotnull(ws_web_page_sk#65) (60) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -(61) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#62] +(61) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#70] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#60] -Right keys [1]: [d_date_sk#62] +Left keys [1]: [ws_sold_date_sk#68] +Right keys [1]: [d_date_sk#70] Join type: Inner Join condition: None (63) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59] -Input [5]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, ws_sold_date_sk#60, d_date_sk#62] +Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] +Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] (unknown) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#63] +Output [1]: [wp_web_page_sk#71] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (65) CometFilter -Input [1]: [wp_web_page_sk#63] -Condition : isnotnull(wp_web_page_sk#63) +Input [1]: [wp_web_page_sk#71] +Condition : isnotnull(wp_web_page_sk#71) (66) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#63] +Input [1]: [wp_web_page_sk#71] (67) BroadcastExchange -Input [1]: [wp_web_page_sk#63] +Input [1]: [wp_web_page_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] (68) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#57] -Right keys [1]: [wp_web_page_sk#63] +Left keys [1]: [ws_web_page_sk#65] +Right keys [1]: [wp_web_page_sk#71] Join type: Inner Join condition: None (69) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] -Input [4]: [ws_web_page_sk#57, ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] +Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] (70) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#58, ws_net_profit#59, wp_web_page_sk#63] -Keys [1]: [wp_web_page_sk#63] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#58)), partial_sum(UnscaledValue(ws_net_profit#59))] -Aggregate Attributes [2]: [sum#64, sum#65] -Results [3]: [wp_web_page_sk#63, sum#66, sum#67] +Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum#72, sum#73] +Results [3]: [wp_web_page_sk#71, sum#74, sum#75] (71) RowToColumnar -Input [3]: [wp_web_page_sk#63, sum#66, sum#67] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] (72) CometColumnarExchange -Input [3]: [wp_web_page_sk#63, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(73) CometHashAggregate -Input [3]: [wp_web_page_sk#63, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#63] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#58)), sum(UnscaledValue(ws_net_profit#59))] +(73) ColumnarToRow [codegen id : 22] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -(74) ColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#63, sales#68, profit#69] +(74) HashAggregate [codegen id : 22] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] +Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] (unknown) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] +Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#73), dynamicpruningexpression(wr_returned_date_sk#73 IN dynamicpruning#74)] +PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (76) CometFilter -Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] -Condition : isnotnull(wr_web_page_sk#70) +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +Condition : isnotnull(wr_web_page_sk#80) (77) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] -(78) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#75] +(78) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#85] (79) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#73] -Right keys [1]: [d_date_sk#75] +Left keys [1]: [wr_returned_date_sk#83] +Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None (80) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72] -Input [5]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wr_returned_date_sk#73, d_date_sk#75] +Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] +Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] (81) ReusedExchange [Reuses operator id: 67] -Output [1]: [wp_web_page_sk#76] +Output [1]: [wp_web_page_sk#86] (82) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#70] -Right keys [1]: [wp_web_page_sk#76] +Left keys [1]: [wr_web_page_sk#80] +Right keys [1]: [wp_web_page_sk#86] Join type: Inner Join condition: None (83) Project [codegen id : 20] -Output [3]: [wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] -Input [4]: [wr_web_page_sk#70, wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] +Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] (84) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#71, wr_net_loss#72, wp_web_page_sk#76] -Keys [1]: [wp_web_page_sk#76] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#71)), partial_sum(UnscaledValue(wr_net_loss#72))] -Aggregate Attributes [2]: [sum#77, sum#78] -Results [3]: [wp_web_page_sk#76, sum#79, sum#80] +Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum#87, sum#88] +Results [3]: [wp_web_page_sk#86, sum#89, sum#90] (85) RowToColumnar -Input [3]: [wp_web_page_sk#76, sum#79, sum#80] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] (86) CometColumnarExchange -Input [3]: [wp_web_page_sk#76, sum#79, sum#80] -Arguments: hashpartitioning(wp_web_page_sk#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(87) CometHashAggregate -Input [3]: [wp_web_page_sk#76, sum#79, sum#80] -Keys [1]: [wp_web_page_sk#76] -Functions [2]: [sum(UnscaledValue(wr_return_amt#71)), sum(UnscaledValue(wr_net_loss#72))] +(87) ColumnarToRow [codegen id : 21] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -(88) ColumnarToRow [codegen id : 21] -Input [3]: [wp_web_page_sk#76, returns#81, profit_loss#82] +(88) HashAggregate [codegen id : 21] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] +Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] (89) BroadcastExchange -Input [3]: [wp_web_page_sk#76, returns#81, profit_loss#82] +Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] (90) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#63] -Right keys [1]: [wp_web_page_sk#76] +Left keys [1]: [wp_web_page_sk#71] +Right keys [1]: [wp_web_page_sk#86] Join type: LeftOuter Join condition: None (91) Project [codegen id : 22] -Output [5]: [web channel AS channel#83, wp_web_page_sk#63 AS id#84, sales#68, coalesce(returns#81, 0.00) AS returns#85, (profit#69 - coalesce(profit_loss#82, 0.00)) AS profit#86] -Input [6]: [wp_web_page_sk#63, sales#68, profit#69, wp_web_page_sk#76, returns#81, profit_loss#82] +Output [5]: [web channel AS channel#95, wp_web_page_sk#71 AS id#96, sales#78, coalesce(returns#93, 0.00) AS returns#97, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#98] +Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] (92) Union (93) HashAggregate [codegen id : 23] -Input [5]: [channel#27, id#28, sales#12, returns#29, profit#30] -Keys [2]: [channel#27, id#28] -Functions [3]: [partial_sum(sales#12), partial_sum(returns#29), partial_sum(profit#30)] -Aggregate Attributes [6]: [sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92] -Results [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Input [5]: [channel#31, id#32, sales#14, returns#33, profit#34] +Keys [2]: [channel#31, id#32] +Functions [3]: [partial_sum(sales#14), partial_sum(returns#33), partial_sum(profit#34)] +Aggregate Attributes [6]: [sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] (94) RowToColumnar -Input [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] (95) CometColumnarExchange -Input [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] -Arguments: hashpartitioning(channel#27, id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(96) CometHashAggregate -Input [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] -Keys [2]: [channel#27, id#28] -Functions [3]: [sum(sales#12), sum(returns#29), sum(profit#30)] - -(97) ReusedExchange [Reuses operator id: 95] -Output [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] - -(98) CometHashAggregate -Input [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] -Keys [2]: [channel#27, id#28] -Functions [3]: [sum(sales#12), sum(returns#29), sum(profit#30)] - -(99) CometHashAggregate -Input [4]: [channel#27, sales#99, returns#100, profit#101] -Keys [1]: [channel#27] -Functions [3]: [partial_sum(sales#99), partial_sum(returns#100), partial_sum(profit#101)] - -(100) CometColumnarExchange -Input [7]: [channel#27, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] -Arguments: hashpartitioning(channel#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(101) CometHashAggregate -Input [7]: [channel#27, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] -Keys [1]: [channel#27] -Functions [3]: [sum(sales#99), sum(returns#100), sum(profit#101)] - -(102) ReusedExchange [Reuses operator id: 95] -Output [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] - -(103) CometHashAggregate -Input [8]: [channel#27, id#28, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] -Keys [2]: [channel#27, id#28] -Functions [3]: [sum(sales#12), sum(returns#29), sum(profit#30)] - -(104) CometHashAggregate -Input [3]: [sales#99, returns#100, profit#101] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(96) ColumnarToRow [codegen id : 24] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(97) HashAggregate [codegen id : 24] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Keys [2]: [channel#31, id#32] +Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] +Results [5]: [channel#31, id#32, cast(sum(sales#14)#111 as decimal(37,2)) AS sales#114, cast(sum(returns#33)#112 as decimal(37,2)) AS returns#115, cast(sum(profit#34)#113 as decimal(38,2)) AS profit#116] + +(98) ReusedExchange [Reuses operator id: 95] +Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(99) ColumnarToRow [codegen id : 48] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(100) HashAggregate [codegen id : 48] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Keys [2]: [channel#31, id#32] +Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] +Results [4]: [channel#31, sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] + +(101) HashAggregate [codegen id : 48] +Input [4]: [channel#31, sales#117, returns#118, profit#119] +Keys [1]: [channel#31] +Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] +Aggregate Attributes [6]: [sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Results [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] + +(102) RowToColumnar +Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] + +(103) CometColumnarExchange +Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(104) ColumnarToRow [codegen id : 49] +Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] + +(105) HashAggregate [codegen id : 49] +Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +Keys [1]: [channel#31] +Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] +Aggregate Attributes [3]: [sum(sales#117)#132, sum(returns#118)#133, sum(profit#119)#134] +Results [5]: [channel#31, null AS id#135, sum(sales#117)#132 AS sales#136, sum(returns#118)#133 AS returns#137, sum(profit#119)#134 AS profit#138] + +(106) ReusedExchange [Reuses operator id: 95] +Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(107) ColumnarToRow [codegen id : 73] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(108) HashAggregate [codegen id : 73] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Keys [2]: [channel#31, id#32] +Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] +Results [3]: [sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] + +(109) HashAggregate [codegen id : 73] +Input [3]: [sales#117, returns#118, profit#119] Keys: [] -Functions [3]: [partial_sum(sales#99), partial_sum(returns#100), partial_sum(profit#101)] +Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] +Aggregate Attributes [6]: [sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Results [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] + +(110) RowToColumnar +Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] -(105) CometColumnarExchange -Input [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +(111) CometColumnarExchange +Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(106) CometHashAggregate -Input [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +(112) ColumnarToRow [codegen id : 74] +Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] + +(113) HashAggregate [codegen id : 74] +Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] Keys: [] -Functions [3]: [sum(sales#99), sum(returns#100), sum(profit#101)] +Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] +Aggregate Attributes [3]: [sum(sales#117)#151, sum(returns#118)#152, sum(profit#119)#153] +Results [5]: [null AS channel#154, null AS id#155, sum(sales#117)#151 AS sales#156, sum(returns#118)#152 AS returns#157, sum(profit#119)#153 AS profit#158] -(107) CometUnion -Child 0 Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] -Child 1 Input [5]: [channel#27, id#117, sales#118, returns#119, profit#120] -Child 2 Input [5]: [channel#121, id#122, sales#123, returns#124, profit#125] +(114) Union -(108) CometHashAggregate -Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] -Keys [5]: [channel#27, id#28, sales#114, returns#115, profit#116] +(115) HashAggregate [codegen id : 75] +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] + +(116) RowToColumnar +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -(109) CometColumnarExchange -Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] -Arguments: hashpartitioning(channel#27, id#28, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(117) CometColumnarExchange +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Arguments: hashpartitioning(channel#31, id#32, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(110) CometHashAggregate -Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] -Keys [5]: [channel#27, id#28, sales#114, returns#115, profit#116] +(118) CometHashAggregate +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] Functions: [] -(111) CometTakeOrderedAndProject -Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,id#28 ASC NULLS FIRST], output=[channel#27,id#28,sales#114,returns#115,profit#116]), 100, [channel#27 ASC NULLS FIRST, id#28 ASC NULLS FIRST], [channel#27, id#28, sales#114, returns#115, profit#116] +(119) CometTakeOrderedAndProject +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#114,returns#115,profit#116]), 100, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#114, returns#115, profit#116] -(112) ColumnarToRow [codegen id : 70] -Input [5]: [channel#27, id#28, sales#114, returns#115, profit#116] +(120) ColumnarToRow [codegen id : 76] +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (125) ++- * ColumnarToRow (124) + +- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#126] +Output [2]: [d_date_sk#6, d_date#159] 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 -(114) CometFilter -Input [2]: [d_date_sk#6, d_date#126] -Condition : (((isnotnull(d_date#126) AND (d_date#126 >= 1998-08-04)) AND (d_date#126 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +(122) CometFilter +Input [2]: [d_date_sk#6, d_date#159] +Condition : (((isnotnull(d_date#159) AND (d_date#159 >= 1998-08-04)) AND (d_date#159 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(115) CometProject -Input [2]: [d_date_sk#6, d_date#126] +(123) CometProject +Input [2]: [d_date_sk#6, d_date#159] Arguments: [d_date_sk#6], [d_date_sk#6] -(116) ColumnarToRow [codegen id : 1] +(124) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(117) BroadcastExchange +(125) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 46 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 46 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 75 Hosting Expression = wr_returned_date_sk#73 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 75 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index 1fc58ce481..d27e3fd80c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -1,170 +1,190 @@ -WholeStageCodegen (70) +WholeStageCodegen (76) ColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] CometColumnarExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [channel,id] #2 - RowToColumnar - WholeStageCodegen (23) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - ColumnarToRow + RowToColumnar + WholeStageCodegen (75) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (24) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (23) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - CometHashAggregate [s_store_sk,sum,sum] - CometColumnarExchange [s_store_sk] #3 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan 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) + Union + WholeStageCodegen (8) + Project [s_store_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_sk] #3 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan 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) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [s_store_sk,sum,sum] - CometColumnarExchange [s_store_sk] #7 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [sr_store_sk] - CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_call_center_sk,sum,sum] - CometColumnarExchange [cs_call_center_sk] #9 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - ColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] - CometColumnarExchange #10 - RowToColumnar - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - ColumnarToRow - InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum] - CometColumnarExchange [wp_web_page_sk] #11 - RowToColumnar - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_web_page_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometColumnarExchange [s_store_sk] #7 + RowToColumnar + WholeStageCodegen (6) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_store_sk] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (14) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum] - CometColumnarExchange [wp_web_page_sk] #14 - RowToColumnar - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [wr_web_page_sk] - CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometColumnarExchange [cs_call_center_sk] #9 + RowToColumnar + WholeStageCodegen (10) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange #10 + RowToColumnar + WholeStageCodegen (13) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [wp_web_page_sk] #12 - CometHashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [channel] #15 - CometHashAggregate [channel,sales,returns,profit] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange #16 - CometHashAggregate [sales,returns,profit] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (22) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [wp_web_page_sk] #11 + RowToColumnar + WholeStageCodegen (17) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (21) + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [wp_web_page_sk] #14 + RowToColumnar + WholeStageCodegen (20) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_web_page_sk] + CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [wp_web_page_sk] #12 + WholeStageCodegen (49) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #15 + RowToColumnar + WholeStageCodegen (48) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (74) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange #16 + RowToColumnar + WholeStageCodegen (73) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index f411b80b79..5e806870bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -1,75 +1,77 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * ColumnarToRow (69) - +- CometSortMergeJoin (68) - :- CometProject (45) - : +- CometSortMergeJoin (44) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometColumnarExchange (19) - : : +- RowToColumnar (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometColumnarExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometColumnarExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (14) - : +- CometSort (43) - : +- CometFilter (42) - : +- CometHashAggregate (41) - : +- CometColumnarExchange (40) - : +- RowToColumnar (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * ColumnarToRow (34) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometColumnarExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (35) - +- CometSort (67) - +- CometFilter (66) - +- CometHashAggregate (65) - +- CometColumnarExchange (64) - +- RowToColumnar (63) - +- * HashAggregate (62) - +- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * ColumnarToRow (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometColumnarExchange (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (54) - : +- CometColumnarExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan parquet spark_catalog.default.catalog_returns (50) - +- ReusedExchange (59) +TakeOrderedAndProject (73) ++- * Project (72) + +- * SortMergeJoin Inner (71) + :- * Project (47) + : +- * SortMergeJoin Inner (46) + : :- * Sort (22) + : : +- * HashAggregate (21) + : : +- * ColumnarToRow (20) + : : +- CometColumnarExchange (19) + : : +- RowToColumnar (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * ColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometColumnarExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometColumnarExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : +- ReusedExchange (14) + : +- * Sort (45) + : +- * Filter (44) + : +- * HashAggregate (43) + : +- * ColumnarToRow (42) + : +- CometColumnarExchange (41) + : +- RowToColumnar (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * ColumnarToRow (35) + : : +- CometProject (34) + : : +- CometFilter (33) + : : +- CometSortMergeJoin (32) + : : :- CometSort (26) + : : : +- CometColumnarExchange (25) + : : : +- CometFilter (24) + : : : +- CometScan parquet spark_catalog.default.web_sales (23) + : : +- CometSort (31) + : : +- CometColumnarExchange (30) + : : +- CometProject (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.web_returns (27) + : +- ReusedExchange (36) + +- * Sort (70) + +- * Filter (69) + +- * HashAggregate (68) + +- * ColumnarToRow (67) + +- CometColumnarExchange (66) + +- RowToColumnar (65) + +- * HashAggregate (64) + +- * Project (63) + +- * BroadcastHashJoin Inner BuildRight (62) + :- * ColumnarToRow (60) + : +- CometProject (59) + : +- CometFilter (58) + : +- CometSortMergeJoin (57) + : :- CometSort (51) + : : +- CometColumnarExchange (50) + : : +- CometFilter (49) + : : +- CometScan parquet spark_catalog.default.catalog_sales (48) + : +- CometSort (56) + : +- CometColumnarExchange (55) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan parquet spark_catalog.default.catalog_returns (52) + +- ReusedExchange (61) (unknown) Scan parquet spark_catalog.default.store_sales @@ -131,7 +133,7 @@ Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, (13) ColumnarToRow [codegen id : 2] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(14) ReusedExchange [Reuses operator id: 75] +(14) ReusedExchange [Reuses operator id: 77] Output [2]: [d_date_sk#12, d_year#13] (15) BroadcastHashJoin [codegen id : 2] @@ -158,245 +160,259 @@ Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) CometHashAggregate +(20) ColumnarToRow [codegen id : 3] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] + +(21) HashAggregate [codegen id : 3] Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] +Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) CometSort -Input [6]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23], [ss_sold_year#20 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] +(22) Sort [codegen id : 3] +Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(23) CometFilter -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Condition : (isnotnull(ws_item_sk#24) AND isnotnull(ws_bill_customer_sk#25)) +(24) CometFilter +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) -(24) CometColumnarExchange -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: hashpartitioning(ws_order_number#26, ws_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(25) CometColumnarExchange +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) CometSort -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_order_number#26 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST] +(26) CometSort +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] +Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(27) CometFilter -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Condition : (isnotnull(wr_order_number#33) AND isnotnull(wr_item_sk#32)) +(28) CometFilter +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) -(28) CometProject -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_item_sk#32, wr_order_number#33] +(29) CometProject +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] -(29) CometColumnarExchange -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: hashpartitioning(wr_order_number#33, wr_item_sk#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(30) CometColumnarExchange +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(30) CometSort -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_order_number#33 ASC NULLS FIRST, wr_item_sk#32 ASC NULLS FIRST] +(31) CometSort +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Right output [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [ws_order_number#26, ws_item_sk#24], [wr_order_number#33, wr_item_sk#32], LeftOuter +(32) CometSortMergeJoin +Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Right output [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter -(32) CometFilter -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Condition : isnull(wr_order_number#33) +(33) CometFilter +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Condition : isnull(wr_order_number#36) -(33) CometProject -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +(34) CometProject +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -(34) ColumnarToRow [codegen id : 4] -Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +(35) ColumnarToRow [codegen id : 5] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -(35) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#35, d_year#36] +(36) ReusedExchange [Reuses operator id: 77] +Output [2]: [d_date_sk#38, d_year#39] -(36) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#35] +(37) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(37) Project [codegen id : 4] -Output [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] -Input [8]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, d_date_sk#35, d_year#36] - -(38) HashAggregate [codegen id : 4] -Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [partial_sum(ws_quantity#27), partial_sum(UnscaledValue(ws_wholesale_cost#28)), partial_sum(UnscaledValue(ws_sales_price#29))] -Aggregate Attributes [3]: [sum#37, sum#38, sum#39] -Results [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] - -(39) RowToColumnar -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] - -(40) CometColumnarExchange -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] -Arguments: hashpartitioning(d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(41) CometHashAggregate -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#40, sum#41, sum#42] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [sum(ws_quantity#27), sum(UnscaledValue(ws_wholesale_cost#28)), sum(UnscaledValue(ws_sales_price#29))] - -(42) CometFilter -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Condition : (coalesce(ws_qty#45, 0) > 0) - -(43) CometSort -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Arguments: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47], [ws_sold_year#43 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST, ws_customer_sk#44 ASC NULLS FIRST] - -(44) CometSortMergeJoin -Left output [6]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Right output [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44], Inner +(38) Project [codegen id : 5] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] + +(39) HashAggregate [codegen id : 5] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum#40, sum#41, sum#42] +Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(40) RowToColumnar +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(41) CometColumnarExchange +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(42) ColumnarToRow [codegen id : 6] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(43) HashAggregate [codegen id : 6] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] +Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] + +(44) Filter [codegen id : 6] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Condition : (coalesce(ws_qty#51, 0) > 0) + +(45) Sort [codegen id : 6] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 + +(46) SortMergeJoin [codegen id : 7] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] +Join type: Inner +Join condition: None -(45) CometProject -Input [12]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47], [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] +(47) Project [codegen id : 7] +Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] +Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#54), dynamicpruningexpression(cs_sold_date_sk#54 IN dynamicpruning#55)] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter -Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Condition : (isnotnull(cs_item_sk#49) AND isnotnull(cs_bill_customer_sk#48)) +(49) CometFilter +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) -(48) CometColumnarExchange -Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: hashpartitioning(cs_order_number#50, cs_item_sk#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(50) CometColumnarExchange +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(49) CometSort -Input [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_order_number#50 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST] +(51) CometSort +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] (unknown) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] +Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(51) CometFilter -Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] -Condition : (isnotnull(cr_order_number#57) AND isnotnull(cr_item_sk#56)) +(53) CometFilter +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) -(52) CometProject -Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_item_sk#56, cr_order_number#57] +(54) CometProject +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] -(53) CometColumnarExchange -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: hashpartitioning(cr_order_number#57, cr_item_sk#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +(55) CometColumnarExchange +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(54) CometSort -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_order_number#57 ASC NULLS FIRST, cr_item_sk#56 ASC NULLS FIRST] +(56) CometSort +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] -(55) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Right output [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cs_order_number#50, cs_item_sk#49], [cr_order_number#57, cr_item_sk#56], LeftOuter +(57) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Right output [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter -(56) CometFilter -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Condition : isnull(cr_order_number#57) +(58) CometFilter +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Condition : isnull(cr_order_number#63) -(57) CometProject -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(59) CometProject +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(58) ColumnarToRow [codegen id : 6] -Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(60) ColumnarToRow [codegen id : 9] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(59) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#59, d_year#60] +(61) ReusedExchange [Reuses operator id: 77] +Output [2]: [d_date_sk#65, d_year#66] -(60) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#54] -Right keys [1]: [d_date_sk#59] +(62) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_sold_date_sk#60] +Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(61) Project [codegen id : 6] -Output [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] -Input [8]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, d_date_sk#59, d_year#60] - -(62) HashAggregate [codegen id : 6] -Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [partial_sum(cs_quantity#51), partial_sum(UnscaledValue(cs_wholesale_cost#52)), partial_sum(UnscaledValue(cs_sales_price#53))] -Aggregate Attributes [3]: [sum#61, sum#62, sum#63] -Results [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] - -(63) RowToColumnar -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] - -(64) CometColumnarExchange -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] -Arguments: hashpartitioning(d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(65) CometHashAggregate -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#64, sum#65, sum#66] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [sum(cs_quantity#51), sum(UnscaledValue(cs_wholesale_cost#52)), sum(UnscaledValue(cs_sales_price#53))] - -(66) CometFilter -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Condition : (coalesce(cs_qty#69, 0) > 0) - -(67) CometSort -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Arguments: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71], [cs_sold_year#67 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST, cs_customer_sk#68 ASC NULLS FIRST] - -(68) CometSortMergeJoin -Left output [9]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] -Right output [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Arguments: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68], Inner - -(69) ColumnarToRow [codegen id : 7] -Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] +(63) Project [codegen id : 9] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] + +(64) HashAggregate [codegen id : 9] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum#67, sum#68, sum#69] +Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] + +(65) RowToColumnar +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] + +(66) CometColumnarExchange +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(67) ColumnarToRow [codegen id : 10] +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] + +(68) HashAggregate [codegen id : 10] +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] +Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] + +(69) Filter [codegen id : 10] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Condition : (coalesce(cs_qty#78, 0) > 0) + +(70) Sort [codegen id : 10] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 + +(71) SortMergeJoin [codegen id : 11] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] +Join type: Inner +Join condition: None -(70) Project [codegen id : 7] -Output [13]: [round((cast(ss_qty#21 as double) / cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)), 2) AS ratio#72, ss_qty#21 AS store_qty#73, ss_wc#22 AS store_wholesale_cost#74, ss_sp#23 AS store_sales_price#75, (coalesce(ws_qty#45, 0) + coalesce(cs_qty#69, 0)) AS other_chan_qty#76, (coalesce(ws_wc#46, 0.00) + coalesce(cs_wc#70, 0.00)) AS other_chan_wholesale_cost#77, (coalesce(ws_sp#47, 0.00) + coalesce(cs_sp#71, 0.00)) AS other_chan_sales_price#78, ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] +(72) Project [codegen id : 11] +Output [13]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -(71) TakeOrderedAndProject -Input [13]: [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78, ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Arguments: 100, [ss_sold_year#20 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#21 DESC NULLS LAST, ss_wc#22 DESC NULLS LAST, ss_sp#23 DESC NULLS LAST, other_chan_qty#76 ASC NULLS FIRST, other_chan_wholesale_cost#77 ASC NULLS FIRST, other_chan_sales_price#78 ASC NULLS FIRST, ratio#72 ASC NULLS FIRST], [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78] +(73) TakeOrderedAndProject +Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Arguments: 100, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (77) ++- * ColumnarToRow (76) + +- CometFilter (75) + +- CometScan parquet spark_catalog.default.date_dim (74) (unknown) Scan parquet spark_catalog.default.date_dim @@ -406,19 +422,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(75) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(74) ColumnarToRow [codegen id : 1] +(76) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(75) BroadcastExchange +(77) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#54 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 48 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index 1b83b3157b..4af11caa06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -1,91 +1,103 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (7) + WholeStageCodegen (11) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - ColumnarToRow + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk] - CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [sr_ticket_number] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk] - CometFilter [ws_qty] - CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + WholeStageCodegen (7) + Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_sold_year,ss_item_sk,ss_customer_sk] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + RowToColumnar + WholeStageCodegen (2) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [sr_ticket_number] + CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometSort [ss_ticket_number,ss_item_sk] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_ticket_number,sr_item_sk] + CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (6) + Sort [ws_sold_year,ws_item_sk,ws_customer_sk] + Filter [ws_qty] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [wr_order_number] + CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometSort [ws_order_number,ws_item_sk] + CometColumnarExchange [ws_order_number,ws_item_sk] #6 + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_order_number,wr_item_sk] + CometColumnarExchange [wr_order_number,wr_item_sk] #7 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (10) + Sort [cs_sold_year,cs_item_sk,cs_customer_sk] + Filter [cs_qty] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 RowToColumnar - WholeStageCodegen (4) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + WholeStageCodegen (9) + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [wr_order_number] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #6 - CometFilter [ws_item_sk,ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cr_order_number] + CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometSort [cs_order_number,cs_item_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #9 + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometSort [wr_order_number,wr_item_sk] - CometColumnarExchange [wr_order_number,wr_item_sk] #7 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometSort [cr_order_number,cr_item_sk] + CometColumnarExchange [cr_order_number,cr_item_sk] #10 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk] - CometFilter [cs_qty] - CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cr_order_number] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #9 - CometFilter [cs_item_sk,cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_order_number,cr_item_sk] - CometColumnarExchange [cr_order_number,cr_item_sk] #10 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 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 71122484d8..f9688675ba 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,129 +1,137 @@ == Physical Plan == -* ColumnarToRow (125) -+- CometTakeOrderedAndProject (124) - +- CometHashAggregate (123) - +- CometColumnarExchange (122) - +- CometHashAggregate (121) - +- CometUnion (120) - :- CometHashAggregate (109) - : +- CometColumnarExchange (108) - : +- RowToColumnar (107) - : +- * HashAggregate (106) - : +- Union (105) - : :- * HashAggregate (40) - : : +- * ColumnarToRow (39) - : : +- CometColumnarExchange (38) - : : +- RowToColumnar (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * Project (21) - : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometColumnarExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometColumnarExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : : : : +- ReusedExchange (13) - : : : : +- BroadcastExchange (19) - : : : : +- * ColumnarToRow (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.store (16) - : : : +- BroadcastExchange (26) - : : : +- * ColumnarToRow (25) - : : : +- CometProject (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.item (22) - : : +- BroadcastExchange (33) - : : +- * ColumnarToRow (32) - : : +- CometProject (31) - : : +- CometFilter (30) - : : +- CometScan parquet spark_catalog.default.promotion (29) - : :- * HashAggregate (72) - : : +- * ColumnarToRow (71) - : : +- CometColumnarExchange (70) - : : +- RowToColumnar (69) - : : +- * HashAggregate (68) - : : +- * Project (67) - : : +- * BroadcastHashJoin Inner BuildRight (66) - : : :- * Project (64) - : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : :- * Project (61) - : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : :- * Project (55) - : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : :- * ColumnarToRow (52) - : : : : : : +- CometProject (51) - : : : : : : +- CometSortMergeJoin (50) - : : : : : : :- CometSort (44) - : : : : : : : +- CometColumnarExchange (43) - : : : : : : : +- CometFilter (42) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- CometSort (49) - : : : : : : +- CometColumnarExchange (48) - : : : : : : +- CometProject (47) - : : : : : : +- CometFilter (46) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) - : : : : : +- ReusedExchange (53) - : : : : +- BroadcastExchange (59) - : : : : +- * ColumnarToRow (58) - : : : : +- CometFilter (57) - : : : : +- CometScan parquet spark_catalog.default.catalog_page (56) - : : : +- ReusedExchange (62) - : : +- ReusedExchange (65) - : +- * HashAggregate (104) - : +- * ColumnarToRow (103) - : +- CometColumnarExchange (102) - : +- RowToColumnar (101) - : +- * HashAggregate (100) - : +- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- * Project (96) - : : +- * BroadcastHashJoin Inner BuildRight (95) - : : :- * Project (93) - : : : +- * BroadcastHashJoin Inner BuildRight (92) - : : : :- * Project (87) - : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : :- * ColumnarToRow (84) - : : : : : +- CometProject (83) - : : : : : +- CometSortMergeJoin (82) - : : : : : :- CometSort (76) - : : : : : : +- CometColumnarExchange (75) - : : : : : : +- CometFilter (74) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (73) - : : : : : +- CometSort (81) - : : : : : +- CometColumnarExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan parquet spark_catalog.default.web_returns (77) - : : : : +- ReusedExchange (85) - : : : +- BroadcastExchange (91) - : : : +- * ColumnarToRow (90) - : : : +- CometFilter (89) - : : : +- CometScan parquet spark_catalog.default.web_site (88) - : : +- ReusedExchange (94) - : +- ReusedExchange (97) - :- CometHashAggregate (114) - : +- CometColumnarExchange (113) - : +- CometHashAggregate (112) - : +- CometHashAggregate (111) - : +- ReusedExchange (110) - +- CometHashAggregate (119) - +- CometColumnarExchange (118) - +- CometHashAggregate (117) - +- CometHashAggregate (116) - +- ReusedExchange (115) +* ColumnarToRow (133) ++- CometTakeOrderedAndProject (132) + +- CometHashAggregate (131) + +- CometColumnarExchange (130) + +- RowToColumnar (129) + +- * HashAggregate (128) + +- Union (127) + :- * HashAggregate (110) + : +- * ColumnarToRow (109) + : +- CometColumnarExchange (108) + : +- RowToColumnar (107) + : +- * HashAggregate (106) + : +- Union (105) + : :- * HashAggregate (40) + : : +- * ColumnarToRow (39) + : : +- CometColumnarExchange (38) + : : +- RowToColumnar (37) + : : +- * HashAggregate (36) + : : +- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Project (28) + : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : :- * Project (21) + : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometColumnarExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometColumnarExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) + : : : : : +- ReusedExchange (13) + : : : : +- BroadcastExchange (19) + : : : : +- * ColumnarToRow (18) + : : : : +- CometFilter (17) + : : : : +- CometScan parquet spark_catalog.default.store (16) + : : : +- BroadcastExchange (26) + : : : +- * ColumnarToRow (25) + : : : +- CometProject (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.item (22) + : : +- BroadcastExchange (33) + : : +- * ColumnarToRow (32) + : : +- CometProject (31) + : : +- CometFilter (30) + : : +- CometScan parquet spark_catalog.default.promotion (29) + : :- * HashAggregate (72) + : : +- * ColumnarToRow (71) + : : +- CometColumnarExchange (70) + : : +- RowToColumnar (69) + : : +- * HashAggregate (68) + : : +- * Project (67) + : : +- * BroadcastHashJoin Inner BuildRight (66) + : : :- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : :- * Project (61) + : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : :- * Project (55) + : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : :- * ColumnarToRow (52) + : : : : : : +- CometProject (51) + : : : : : : +- CometSortMergeJoin (50) + : : : : : : :- CometSort (44) + : : : : : : : +- CometColumnarExchange (43) + : : : : : : : +- CometFilter (42) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) + : : : : : : +- CometSort (49) + : : : : : : +- CometColumnarExchange (48) + : : : : : : +- CometProject (47) + : : : : : : +- CometFilter (46) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) + : : : : : +- ReusedExchange (53) + : : : : +- BroadcastExchange (59) + : : : : +- * ColumnarToRow (58) + : : : : +- CometFilter (57) + : : : : +- CometScan parquet spark_catalog.default.catalog_page (56) + : : : +- ReusedExchange (62) + : : +- ReusedExchange (65) + : +- * HashAggregate (104) + : +- * ColumnarToRow (103) + : +- CometColumnarExchange (102) + : +- RowToColumnar (101) + : +- * HashAggregate (100) + : +- * Project (99) + : +- * BroadcastHashJoin Inner BuildRight (98) + : :- * Project (96) + : : +- * BroadcastHashJoin Inner BuildRight (95) + : : :- * Project (93) + : : : +- * BroadcastHashJoin Inner BuildRight (92) + : : : :- * Project (87) + : : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : : :- * ColumnarToRow (84) + : : : : : +- CometProject (83) + : : : : : +- CometSortMergeJoin (82) + : : : : : :- CometSort (76) + : : : : : : +- CometColumnarExchange (75) + : : : : : : +- CometFilter (74) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (73) + : : : : : +- CometSort (81) + : : : : : +- CometColumnarExchange (80) + : : : : : +- CometProject (79) + : : : : : +- CometFilter (78) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (77) + : : : : +- ReusedExchange (85) + : : : +- BroadcastExchange (91) + : : : +- * ColumnarToRow (90) + : : : +- CometFilter (89) + : : : +- CometScan parquet spark_catalog.default.web_site (88) + : : +- ReusedExchange (94) + : +- ReusedExchange (97) + :- * HashAggregate (118) + : +- * ColumnarToRow (117) + : +- CometColumnarExchange (116) + : +- RowToColumnar (115) + : +- * HashAggregate (114) + : +- * HashAggregate (113) + : +- * ColumnarToRow (112) + : +- ReusedExchange (111) + +- * HashAggregate (126) + +- * ColumnarToRow (125) + +- CometColumnarExchange (124) + +- RowToColumnar (123) + +- * HashAggregate (122) + +- * HashAggregate (121) + +- * ColumnarToRow (120) + +- ReusedExchange (119) (unknown) Scan parquet spark_catalog.default.store_sales @@ -181,7 +189,7 @@ Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss (12) ColumnarToRow [codegen id : 5] Input [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(13) ReusedExchange [Reuses operator id: 130] +(13) ReusedExchange [Reuses operator id: 138] Output [1]: [d_date_sk#14] (14) BroadcastHashJoin [codegen id : 5] @@ -365,7 +373,7 @@ Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_p (52) ColumnarToRow [codegen id : 11] Input [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -(53) ReusedExchange [Reuses operator id: 130] +(53) ReusedExchange [Reuses operator id: 138] Output [1]: [d_date_sk#52] (54) BroadcastHashJoin [codegen id : 11] @@ -511,7 +519,7 @@ Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price (84) ColumnarToRow [codegen id : 17] Input [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -(85) ReusedExchange [Reuses operator id: 130] +(85) ReusedExchange [Reuses operator id: 138] Output [1]: [d_date_sk#88] (86) BroadcastHashJoin [codegen id : 17] @@ -618,110 +626,147 @@ Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#1 Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(109) CometHashAggregate +(109) ColumnarToRow [codegen id : 20] +Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] + +(110) HashAggregate [codegen id : 20] Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] +Results [5]: [channel#34, id#35, cast(sum(sales#36)#123 as decimal(37,2)) AS sales#126, cast(sum(returns#37)#124 as decimal(38,2)) AS returns#127, cast(sum(profit#38)#125 as decimal(38,2)) AS profit#128] -(110) ReusedExchange [Reuses operator id: 108] +(111) ReusedExchange [Reuses operator id: 108] Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -(111) CometHashAggregate +(112) ColumnarToRow [codegen id : 40] +Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] + +(113) HashAggregate [codegen id : 40] Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] +Results [4]: [channel#34, sum(sales#36)#123 AS sales#129, sum(returns#37)#124 AS returns#130, sum(profit#38)#125 AS profit#131] -(112) CometHashAggregate -Input [4]: [channel#34, sales#123, returns#124, profit#125] +(114) HashAggregate [codegen id : 40] +Input [4]: [channel#34, sales#129, returns#130, profit#131] Keys [1]: [channel#34] -Functions [3]: [partial_sum(sales#123), partial_sum(returns#124), partial_sum(profit#125)] +Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] +Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] +Results [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(115) RowToColumnar +Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -(113) CometColumnarExchange -Input [7]: [channel#34, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +(116) CometColumnarExchange +Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(114) CometHashAggregate -Input [7]: [channel#34, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +(117) ColumnarToRow [codegen id : 41] +Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(118) HashAggregate [codegen id : 41] +Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Keys [1]: [channel#34] -Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] +Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] +Aggregate Attributes [3]: [sum(sales#129)#144, sum(returns#130)#145, sum(profit#131)#146] +Results [5]: [channel#34, null AS id#147, sum(sales#129)#144 AS sales#148, sum(returns#130)#145 AS returns#149, sum(profit#131)#146 AS profit#150] -(115) ReusedExchange [Reuses operator id: 108] +(119) ReusedExchange [Reuses operator id: 108] Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -(116) CometHashAggregate +(120) ColumnarToRow [codegen id : 61] +Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] + +(121) HashAggregate [codegen id : 61] Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] +Results [3]: [sum(sales#36)#123 AS sales#129, sum(returns#37)#124 AS returns#130, sum(profit#38)#125 AS profit#131] -(117) CometHashAggregate -Input [3]: [sales#123, returns#124, profit#125] +(122) HashAggregate [codegen id : 61] +Input [3]: [sales#129, returns#130, profit#131] Keys: [] -Functions [3]: [partial_sum(sales#123), partial_sum(returns#124), partial_sum(profit#125)] +Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] +Aggregate Attributes [6]: [sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] +Results [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] + +(123) RowToColumnar +Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -(118) CometColumnarExchange -Input [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] +(124) CometColumnarExchange +Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] -(119) CometHashAggregate -Input [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] +(125) ColumnarToRow [codegen id : 62] +Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] + +(126) HashAggregate [codegen id : 62] +Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Keys: [] -Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] +Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] +Aggregate Attributes [3]: [sum(sales#129)#163, sum(returns#130)#164, sum(profit#131)#165] +Results [5]: [null AS channel#166, null AS id#167, sum(sales#129)#163 AS sales#168, sum(returns#130)#164 AS returns#169, sum(profit#131)#165 AS profit#170] -(120) CometUnion -Child 0 Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] -Child 1 Input [5]: [channel#34, id#141, sales#142, returns#143, profit#144] -Child 2 Input [5]: [channel#145, id#146, sales#147, returns#148, profit#149] +(127) Union -(121) CometHashAggregate -Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] -Keys [5]: [channel#34, id#35, sales#138, returns#139, profit#140] +(128) HashAggregate [codegen id : 63] +Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] + +(129) RowToColumnar +Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -(122) CometColumnarExchange -Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] -Arguments: hashpartitioning(channel#34, id#35, sales#138, returns#139, profit#140, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] +(130) CometColumnarExchange +Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Arguments: hashpartitioning(channel#34, id#35, sales#126, returns#127, profit#128, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(123) CometHashAggregate -Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] -Keys [5]: [channel#34, id#35, sales#138, returns#139, profit#140] +(131) CometHashAggregate +Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Functions: [] -(124) CometTakeOrderedAndProject -Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#34 ASC NULLS FIRST,id#35 ASC NULLS FIRST], output=[channel#34,id#35,sales#138,returns#139,profit#140]), 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#138, returns#139, profit#140] +(132) CometTakeOrderedAndProject +Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#34 ASC NULLS FIRST,id#35 ASC NULLS FIRST], output=[channel#34,id#35,sales#126,returns#127,profit#128]), 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#126, returns#127, profit#128] -(125) ColumnarToRow [codegen id : 58] -Input [5]: [channel#34, id#35, sales#138, returns#139, profit#140] +(133) ColumnarToRow [codegen id : 64] +Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (130) -+- * ColumnarToRow (129) - +- CometProject (128) - +- CometFilter (127) - +- CometScan parquet spark_catalog.default.date_dim (126) +BroadcastExchange (138) ++- * ColumnarToRow (137) + +- CometProject (136) + +- CometFilter (135) + +- CometScan parquet spark_catalog.default.date_dim (134) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#150] +Output [2]: [d_date_sk#14, d_date#171] 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 -(127) CometFilter -Input [2]: [d_date_sk#14, d_date#150] -Condition : (((isnotnull(d_date#150) AND (d_date#150 >= 1998-08-04)) AND (d_date#150 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +(135) CometFilter +Input [2]: [d_date_sk#14, d_date#171] +Condition : (((isnotnull(d_date#171) AND (d_date#171 >= 1998-08-04)) AND (d_date#171 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(128) CometProject -Input [2]: [d_date_sk#14, d_date#150] +(136) CometProject +Input [2]: [d_date_sk#14, d_date#171] Arguments: [d_date_sk#14], [d_date_sk#14] -(129) ColumnarToRow [codegen id : 1] +(137) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(130) BroadcastExchange +(138) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] 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 f694eedfaf..cd7515a4d5 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,173 +1,193 @@ -WholeStageCodegen (58) +WholeStageCodegen (64) ColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] CometColumnarExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [channel,id] #2 - RowToColumnar - WholeStageCodegen (19) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - 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_ticket_number,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_ticket_number] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] - CometScan 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) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number] - CometColumnarExchange [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] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow + RowToColumnar + WholeStageCodegen (63) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (20) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + RowToColumnar + WholeStageCodegen (19) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + RowToColumnar + WholeStageCodegen (5) + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + 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_ticket_number,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_ticket_number] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometScan 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) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number] + CometColumnarExchange [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] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (12) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #10 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - 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_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometColumnarExchange [cs_item_sk,cs_order_number] #11 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - CometScan 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] - CometColumnarExchange [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] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + BroadcastExchange #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - ColumnarToRow + BroadcastExchange #9 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_channel_tv,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (12) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #10 + RowToColumnar + WholeStageCodegen (11) + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + 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_item_sk,cs_order_number,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number] + CometColumnarExchange [cs_item_sk,cs_order_number] #11 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometScan 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] + CometColumnarExchange [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] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (18) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #14 - RowToColumnar - WholeStageCodegen (17) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - 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_order_number,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_order_number] - CometColumnarExchange [ws_item_sk,ws_order_number] #15 - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - CometScan 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] - CometColumnarExchange [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] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [i_item_sk] #8 InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (14) - ColumnarToRow + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (18) + 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] + ColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #14 + RowToColumnar + WholeStageCodegen (17) + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + 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_order_number,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_order_number] + CometColumnarExchange [ws_item_sk,ws_order_number] #15 + CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometScan 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] + CometColumnarExchange [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] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (14) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange [channel] #18 - CometHashAggregate [channel,sales,returns,profit] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarExchange #19 - CometHashAggregate [sales,returns,profit] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + ReusedExchange [i_item_sk] #8 + InputAdapter + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (41) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [channel] #18 + RowToColumnar + WholeStageCodegen (40) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (62) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange #19 + RowToColumnar + WholeStageCodegen (61) + 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] + ColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt index 2c60857ef7..c9db800a80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt @@ -1,40 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (36) -+- * Project (35) - +- Window (34) - +- * ColumnarToRow (33) - +- CometSort (32) - +- CometColumnarExchange (31) - +- CometHashAggregate (30) - +- CometColumnarExchange (29) - +- CometHashAggregate (28) - +- CometUnion (27) - :- CometHashAggregate (16) - : +- CometColumnarExchange (15) - : +- RowToColumnar (14) - : +- * HashAggregate (13) - : +- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - :- CometHashAggregate (21) - : +- CometColumnarExchange (20) - : +- CometHashAggregate (19) - : +- CometHashAggregate (18) - : +- ReusedExchange (17) - +- CometHashAggregate (26) - +- CometColumnarExchange (25) - +- CometHashAggregate (24) - +- CometHashAggregate (23) - +- ReusedExchange (22) +TakeOrderedAndProject (44) ++- * Project (43) + +- Window (42) + +- * ColumnarToRow (41) + +- CometSort (40) + +- CometColumnarExchange (39) + +- CometHashAggregate (38) + +- CometColumnarExchange (37) + +- RowToColumnar (36) + +- * HashAggregate (35) + +- Union (34) + :- * HashAggregate (17) + : +- * ColumnarToRow (16) + : +- CometColumnarExchange (15) + : +- RowToColumnar (14) + : +- * HashAggregate (13) + : +- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + :- * HashAggregate (25) + : +- * ColumnarToRow (24) + : +- CometColumnarExchange (23) + : +- RowToColumnar (22) + : +- * HashAggregate (21) + : +- * HashAggregate (20) + : +- * ColumnarToRow (19) + : +- ReusedExchange (18) + +- * HashAggregate (33) + +- * ColumnarToRow (32) + +- CometColumnarExchange (31) + +- RowToColumnar (30) + +- * HashAggregate (29) + +- * HashAggregate (28) + +- * ColumnarToRow (27) + +- ReusedExchange (26) (unknown) Scan parquet spark_catalog.default.web_sales @@ -52,7 +60,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 41] +(4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 3] @@ -107,126 +115,163 @@ Input [3]: [i_category#8, i_class#7, sum#10] Input [3]: [i_category#8, i_class#7, sum#10] Arguments: hashpartitioning(i_category#8, i_class#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 4] +Input [3]: [i_category#8, i_class#7, sum#10] + +(17) HashAggregate [codegen id : 4] Input [3]: [i_category#8, i_class#7, sum#10] Keys [2]: [i_category#8, i_class#7] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) as decimal(27,2)) AS total_sum#12, i_category#8, i_class#7, 0 AS g_category#13, 0 AS g_class#14, 0 AS lochierarchy#15] + +(18) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#8, i_class#7, sum#16] -(17) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#8, i_class#7, sum#11] +(19) ColumnarToRow [codegen id : 8] +Input [3]: [i_category#8, i_class#7, sum#16] -(18) CometHashAggregate -Input [3]: [i_category#8, i_class#7, sum#11] +(20) HashAggregate [codegen id : 8] +Input [3]: [i_category#8, i_class#7, sum#16] Keys [2]: [i_category#8, i_class#7] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) AS total_sum#17, i_category#8] -(19) CometHashAggregate -Input [2]: [total_sum#12, i_category#8] +(21) HashAggregate [codegen id : 8] +Input [2]: [total_sum#17, i_category#8] Keys [1]: [i_category#8] -Functions [1]: [partial_sum(total_sum#12)] +Functions [1]: [partial_sum(total_sum#17)] +Aggregate Attributes [2]: [sum#18, isEmpty#19] +Results [3]: [i_category#8, sum#20, isEmpty#21] -(20) CometColumnarExchange -Input [3]: [i_category#8, sum#13, isEmpty#14] +(22) RowToColumnar +Input [3]: [i_category#8, sum#20, isEmpty#21] + +(23) CometColumnarExchange +Input [3]: [i_category#8, sum#20, isEmpty#21] Arguments: hashpartitioning(i_category#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometHashAggregate -Input [3]: [i_category#8, sum#13, isEmpty#14] +(24) ColumnarToRow [codegen id : 9] +Input [3]: [i_category#8, sum#20, isEmpty#21] + +(25) HashAggregate [codegen id : 9] +Input [3]: [i_category#8, sum#20, isEmpty#21] Keys [1]: [i_category#8] -Functions [1]: [sum(total_sum#12)] +Functions [1]: [sum(total_sum#17)] +Aggregate Attributes [1]: [sum(total_sum#17)#22] +Results [6]: [sum(total_sum#17)#22 AS total_sum#23, i_category#8, null AS i_class#24, 0 AS g_category#25, 1 AS g_class#26, 1 AS lochierarchy#27] + +(26) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#8, i_class#7, sum#28] -(22) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#8, i_class#7, sum#15] +(27) ColumnarToRow [codegen id : 13] +Input [3]: [i_category#8, i_class#7, sum#28] -(23) CometHashAggregate -Input [3]: [i_category#8, i_class#7, sum#15] +(28) HashAggregate [codegen id : 13] +Input [3]: [i_category#8, i_class#7, sum#28] Keys [2]: [i_category#8, i_class#7] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) AS total_sum#17] -(24) CometHashAggregate -Input [1]: [total_sum#12] +(29) HashAggregate [codegen id : 13] +Input [1]: [total_sum#17] Keys: [] -Functions [1]: [partial_sum(total_sum#12)] +Functions [1]: [partial_sum(total_sum#17)] +Aggregate Attributes [2]: [sum#29, isEmpty#30] +Results [2]: [sum#31, isEmpty#32] -(25) CometColumnarExchange -Input [2]: [sum#16, isEmpty#17] +(30) RowToColumnar +Input [2]: [sum#31, isEmpty#32] + +(31) CometColumnarExchange +Input [2]: [sum#31, isEmpty#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) CometHashAggregate -Input [2]: [sum#16, isEmpty#17] +(32) ColumnarToRow [codegen id : 14] +Input [2]: [sum#31, isEmpty#32] + +(33) HashAggregate [codegen id : 14] +Input [2]: [sum#31, isEmpty#32] Keys: [] -Functions [1]: [sum(total_sum#12)] +Functions [1]: [sum(total_sum#17)] +Aggregate Attributes [1]: [sum(total_sum#17)#33] +Results [6]: [sum(total_sum#17)#33 AS total_sum#34, null AS i_category#35, null AS i_class#36, 1 AS g_category#37, 1 AS g_class#38, 2 AS lochierarchy#39] -(27) CometUnion -Child 0 Input [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] -Child 1 Input [6]: [total_sum#22, i_category#8, i_class#23, g_category#24, g_class#25, lochierarchy#26] -Child 2 Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] +(34) Union -(28) CometHashAggregate -Input [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] -Keys [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] +(35) HashAggregate [codegen id : 15] +Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] Functions: [] +Aggregate Attributes: [] +Results [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -(29) CometColumnarExchange -Input [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] -Arguments: hashpartitioning(total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(36) RowToColumnar +Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -(30) CometHashAggregate -Input [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] -Keys [6]: [total_sum#18, i_category#8, i_class#7, g_category#19, g_class#20, lochierarchy#21] +(37) CometColumnarExchange +Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +Arguments: hashpartitioning(total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(38) CometHashAggregate +Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] Functions: [] -(31) CometColumnarExchange -Input [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33] -Arguments: hashpartitioning(lochierarchy#21, _w0#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(39) CometColumnarExchange +Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] +Arguments: hashpartitioning(lochierarchy#15, _w0#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(32) CometSort -Input [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33] -Arguments: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33], [lochierarchy#21 ASC NULLS FIRST, _w0#33 ASC NULLS FIRST, total_sum#18 DESC NULLS LAST] +(40) CometSort +Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] +Arguments: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40], [lochierarchy#15 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#12 DESC NULLS LAST] -(33) ColumnarToRow [codegen id : 10] -Input [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33] +(41) ColumnarToRow [codegen id : 16] +Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] -(34) Window -Input [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33] -Arguments: [rank(total_sum#18) windowspecdefinition(lochierarchy#21, _w0#33, total_sum#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [lochierarchy#21, _w0#33], [total_sum#18 DESC NULLS LAST] +(42) Window +Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] +Arguments: [rank(total_sum#12) windowspecdefinition(lochierarchy#15, _w0#40, total_sum#12 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [lochierarchy#15, _w0#40], [total_sum#12 DESC NULLS LAST] -(35) Project [codegen id : 11] -Output [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, rank_within_parent#34] -Input [6]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, _w0#33, rank_within_parent#34] +(43) Project [codegen id : 17] +Output [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] +Input [6]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40, rank_within_parent#41] -(36) TakeOrderedAndProject -Input [5]: [total_sum#18, i_category#8, i_class#7, lochierarchy#21, rank_within_parent#34] -Arguments: 100, [lochierarchy#21 DESC NULLS LAST, CASE WHEN (lochierarchy#21 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#18, i_category#8, i_class#7, lochierarchy#21, rank_within_parent#34] +(44) TakeOrderedAndProject +Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] +Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (41) -+- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#35] +Output [2]: [d_date_sk#5, d_month_seq#42] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(38) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#35] -Condition : (((isnotnull(d_month_seq#35) AND (d_month_seq#35 >= 1212)) AND (d_month_seq#35 <= 1223)) AND isnotnull(d_date_sk#5)) +(46) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#42] +Condition : (((isnotnull(d_month_seq#42) AND (d_month_seq#42 >= 1212)) AND (d_month_seq#42 <= 1223)) AND isnotnull(d_date_sk#5)) -(39) CometProject -Input [2]: [d_date_sk#5, d_month_seq#35] +(47) CometProject +Input [2]: [d_date_sk#5, d_month_seq#42] Arguments: [d_date_sk#5], [d_date_sk#5] -(40) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(41) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index ef041fca19..6abc26081c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -1,54 +1,74 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (11) + WholeStageCodegen (17) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (10) + WholeStageCodegen (16) ColumnarToRow InputAdapter CometSort [lochierarchy,_w0,total_sum] CometColumnarExchange [lochierarchy,_w0] #1 CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion - CometHashAggregate [i_category,i_class,sum] - CometColumnarExchange [i_category,i_class] #3 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) + RowToColumnar + WholeStageCodegen (15) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (4) + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,g_category,g_class,lochierarchy,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #3 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] + Project [ws_net_paid,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [i_category,sum,isEmpty] - CometColumnarExchange [i_category] #6 - CometHashAggregate [i_category,total_sum] - CometHashAggregate [i_category,i_class,sum] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] - CometColumnarExchange #7 - CometHashAggregate [total_sum] - CometHashAggregate [i_category,i_class,sum] - ReusedExchange [i_category,i_class,sum] #3 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (9) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #6 + RowToColumnar + WholeStageCodegen (8) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 + WholeStageCodegen (14) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + ColumnarToRow + InputAdapter + CometColumnarExchange #7 + RowToColumnar + WholeStageCodegen (13) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + ColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index 080613d65b..6553eeabe7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -1,29 +1,31 @@ == Physical Plan == -* ColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- RowToColumnar (22) - +- * Project (21) - +- Window (20) - +- * ColumnarToRow (19) - +- CometSort (18) - +- CometColumnarExchange (17) - +- CometHashAggregate (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +* ColumnarToRow (27) ++- CometSort (26) + +- CometColumnarExchange (25) + +- RowToColumnar (24) + +- * Project (23) + +- Window (22) + +- * ColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- RowToColumnar (18) + +- * HashAggregate (17) + +- * ColumnarToRow (16) + +- CometColumnarExchange (15) + +- RowToColumnar (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.store_sales @@ -69,7 +71,7 @@ Join condition: None Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 30] +(10) ReusedExchange [Reuses operator id: 32] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -96,73 +98,81 @@ Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate +(16) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] +Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16] + +(18) RowToColumnar +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -(17) CometColumnarExchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +(19) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) CometSort -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] -Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15], [i_class#9 ASC NULLS FIRST] +(20) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16], [i_class#9 ASC NULLS FIRST] -(19) ColumnarToRow [codegen id : 4] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] +(21) ColumnarToRow [codegen id : 5] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -(20) Window -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15] -Arguments: [sum(_w0#15) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#16], [i_class#9] +(22) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(21) Project [codegen id : 5] -Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, ((_w0#15 * 100) / _we0#16) AS revenueratio#17] -Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, _w0#15, _we0#16] +(23) Project [codegen id : 6] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] -(22) RowToColumnar -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +(24) RowToColumnar +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -(23) CometColumnarExchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] -Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(25) CometColumnarExchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometSort -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] -Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17], [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#17 ASC NULLS FIRST] +(26) CometSort +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18], [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 6] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#14, revenueratio#17] +(27) ColumnarToRow [codegen id : 7] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * ColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.date_dim (26) +BroadcastExchange (32) ++- * ColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan parquet spark_catalog.default.date_dim (28) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#18] +Output [2]: [d_date_sk#11, d_date#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter -Input [2]: [d_date_sk#11, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-22)) AND (d_date#18 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(29) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(28) CometProject -Input [2]: [d_date_sk#11, d_date#18] +(30) CometProject +Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(29) ColumnarToRow [codegen id : 1] +(31) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(30) BroadcastExchange +(32) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index b59521b6c2..ef8abe2979 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -1,45 +1,49 @@ -WholeStageCodegen (6) +WholeStageCodegen (7) ColumnarToRow InputAdapter CometSort [i_category,i_class,i_item_id,i_item_desc,revenueratio] CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 RowToColumnar - WholeStageCodegen (5) + WholeStageCodegen (6) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (4) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometSort [i_class] CometColumnarExchange [i_class] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 + RowToColumnar + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + ColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + RowToColumnar + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 From 343dabe7ca2096b725af153f4432a48133b96725 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 14 Apr 2024 16:02:48 -0700 Subject: [PATCH 07/30] Update diff --- dev/diffs/3.4.2.diff | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 5d3eb790e8..6fb3f4efd0 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -453,6 +453,20 @@ index b5b34922694..a72403780c4 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +index 525d97e4998..8a3e7457618 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +@@ -1508,7 +1508,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark + checkAnswer(sql("select -0.001"), Row(BigDecimal("-0.001"))) + } + +- test("external sorting updates peak execution memory") { ++ test("external sorting updates peak execution memory", ++ IgnoreComet("TODO: native CometSort does not update peak execution memory")) { + AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "external sort") { + sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC").collect() + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 3cfda19134a..278bb1060c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala From 8f0875fee3adc96aea37d2b89186c005a29a35ff Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 14 Apr 2024 17:20:36 -0700 Subject: [PATCH 08/30] Update more diff --- dev/diffs/3.4.2.diff | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 6fb3f4efd0..79609f65ee 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -571,6 +571,21 @@ index 418ca3430bb..eb8267192f8 100644 Seq("json", "orc", "parquet").foreach { format => withTempPath { path => val dir = path.getCanonicalPath +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala +index 743ec41dbe7..9f30d6c8e04 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala +@@ -53,6 +53,10 @@ class LogicalPlanTagInSparkPlanSuite extends TPCDSQuerySuite with DisableAdaptiv + case ColumnarToRowExec(i: InputAdapter) => isScanPlanTree(i.child) + case p: ProjectExec => isScanPlanTree(p.child) + case f: FilterExec => isScanPlanTree(f.child) ++ // Comet produces scan plan tree like: ++ // ColumnarToRow ++ // +- ReusedExchange ++ case _: ReusedExchangeExec => false + case _: LeafExecNode => true + case _ => false + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala index 9e9d717db3b..91a4f9a38d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala From 2d6a7eae6a091fad482d407f80e9b61e1df4eeb6 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 15 Apr 2024 14:59:48 -0700 Subject: [PATCH 09/30] Update DataFusion commit --- core/Cargo.lock | 196 +++++++++++++++++++++++------------------------- 1 file changed, 92 insertions(+), 104 deletions(-) diff --git a/core/Cargo.lock b/core/Cargo.lock index 3fb7b5f62e..52f105591d 100644 --- a/core/Cargo.lock +++ b/core/Cargo.lock @@ -57,9 +57,9 @@ dependencies = [ [[package]] name = "allocator-api2" -version = "0.2.16" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0942ffc6dcaadf03badf6e6a2d0228460359d5e34b57ccdc720b7382dfbd5ec5" +checksum = "5c6cb57a04249c6480766f7f7cef5467412af1490f8d1e243141daddada3264f" [[package]] name = "android-tzdata" @@ -90,9 +90,9 @@ checksum = "8901269c6307e8d93993578286ac0edf7f195079ffff5ebdeea6a59ffb7e36bc" [[package]] name = "anyhow" -version = "1.0.81" +version = "1.0.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0952808a6c2afd1aa8947271f3a60f1a6763c7b912d210184c5149b5cf147247" +checksum = "f538837af36e6f6a9be0faa67f9a314f8119e4e4b5867c6ab40ed60360142519" [[package]] name = "arc-swap" @@ -327,13 +327,13 @@ checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131" [[package]] name = "async-trait" -version = "0.1.79" +version = "0.1.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a507401cad91ec6a857ed5513a2073c82a9b9048762b885bb98655b306964681" +checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" dependencies = [ "proc-macro2", "quote", - "syn 2.0.57", + "syn 2.0.59", ] [[package]] @@ -438,9 +438,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.15.4" +version = "3.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ff69b9dd49fd426c69a0db9fc04dd934cdb6645ff000864d98f7e2af8830eaa" +checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" [[package]] name = "bytemuck" @@ -468,9 +468,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.0.90" +version = "1.0.94" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cd6604a82acf3039f1144f54b8eb34e91ffba622051189e71b781822d5ee1f5" +checksum = "17f6e324229dc011159fcc089755d1e2e216a90d43a7dea6853ca740b84f35e7" dependencies = [ "jobserver", "libc", @@ -490,14 +490,14 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "chrono" -version = "0.4.37" +version = "0.4.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a0d04d43504c61aa6c7531f1871dd0d418d91130162063b789da00fd7057a5e" +checksum = "a21f936df1771bf62b77f047b726c4625ff2e8aa607c01ec06e5a05bd8463401" dependencies = [ "android-tzdata", "iana-time-zone", "num-traits", - "windows-targets 0.52.4", + "windows-targets 0.52.5", ] [[package]] @@ -576,9 +576,9 @@ checksum = "98cc8fbded0c607b7ba9dd60cd98df59af97e84d24e49c8557331cfc26d301ce" [[package]] name = "combine" -version = "4.6.6" +version = "4.6.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35ed6e9d84f0b51a7f52daf1c7d71dd136fd7a3f41a8462b8cdb8c78d920fad4" +checksum = "ba5a308b75df32fe02788e748662718f03fde005016435c444eea572398219fd" dependencies = [ "bytes", "memchr", @@ -625,7 +625,7 @@ dependencies = [ "parquet-format", "paste", "pprof", - "prost 0.12.3", + "prost 0.12.4", "prost-build", "rand", "regex", @@ -643,12 +643,12 @@ dependencies = [ [[package]] name = "comfy-table" -version = "7.1.0" +version = "7.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" +checksum = "b34115915337defe99b2aff5c2ce6771e5fbc4079f4b506301f5cf394c8452f7" dependencies = [ - "strum 0.25.0", - "strum_macros 0.25.3", + "strum", + "strum_macros", "unicode-width", ] @@ -923,8 +923,8 @@ dependencies = [ "datafusion-common", "paste", "sqlparser", - "strum 0.26.2", - "strum_macros 0.26.2", + "strum", + "strum_macros", ] [[package]] @@ -1044,7 +1044,7 @@ dependencies = [ "datafusion-expr", "log", "sqlparser", - "strum 0.26.2", + "strum", ] [[package]] @@ -1092,9 +1092,9 @@ checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" [[package]] name = "either" -version = "1.10.0" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11157ac094ffbdde99aa67b23417ebdd801842852b500e395a45a9c0aac03e4a" +checksum = "a47c1c47d2f5964e29c61246e81db715514cd532db6b5116a25ea3c03d6780a2" [[package]] name = "equivalent" @@ -1227,7 +1227,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.57", + "syn 2.0.59", ] [[package]] @@ -1272,9 +1272,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.12" +version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "190092ea657667030ac6a35e305e62fc4dd69fd98ac98631e5d3a2b1575a12b5" +checksum = "94b22e06ecb0110981051723910cbf0b5f5e09a2062dd7663334ee79a9d1286c" dependencies = [ "cfg-if", "libc", @@ -1526,9 +1526,9 @@ checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" [[package]] name = "jobserver" -version = "0.1.28" +version = "0.1.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab46a6e9526ddef3ae7f787c06f0f2600639ba80ea3eade3d8e670a2230f51d6" +checksum = "685a7d121ee3f65ae4fddd72b25a04bb36b6af81bc0828f7d5434c0fe60fa3a2" dependencies = [ "libc", ] @@ -1794,9 +1794,9 @@ dependencies = [ [[package]] name = "num" -version = "0.4.1" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b05180d69e3da0e530ba2a1dae5110317e49e3b7f3d41be227dc5f92e49ee7af" +checksum = "3135b08af27d103b0a51f2ae0f8632117b7b185ccf931445affa8df530576a41" dependencies = [ "num-bigint", "num-complex", @@ -2142,9 +2142,9 @@ checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" [[package]] name = "proc-macro2" -version = "1.0.79" +version = "1.0.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e835ff2298f5721608eb1a980ecaee1aef2c132bf95ecc026a11b7bf3c01c02e" +checksum = "a56dea16b0a29e94408b9aa5e2940a4eedbd128a1ba20e8f7ae60fd3d465af0e" dependencies = [ "unicode-ident", ] @@ -2161,12 +2161,12 @@ dependencies = [ [[package]] name = "prost" -version = "0.12.3" +version = "0.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "146c289cda302b98a28d40c8b3b90498d6e526dd24ac2ecea73e4e491685b94a" +checksum = "d0f5d036824e4761737860779c906171497f6d55681139d8312388f8fe398922" dependencies = [ "bytes", - "prost-derive 0.12.3", + "prost-derive 0.12.4", ] [[package]] @@ -2204,15 +2204,15 @@ dependencies = [ [[package]] name = "prost-derive" -version = "0.12.3" +version = "0.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efb6c9a1dd1def8e2124d17e83a20af56f1570d6c2d2bd9e266ccb768df3840e" +checksum = "19de2de2a00075bf566bee3bd4db014b11587e84184d3f7a791bc17f1a8e9e48" dependencies = [ "anyhow", - "itertools 0.11.0", + "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.57", + "syn 2.0.59", ] [[package]] @@ -2236,9 +2236,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.35" +version = "1.0.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "291ec9ab5efd934aaf503a6466c5d5251535d108ee747472c3977cc5acc868ef" +checksum = "0fa76aaf39101c457836aec0ce2316dbdc3ab723cdda1c6bd4e6ad4208acaca7" dependencies = [ "proc-macro2", ] @@ -2370,9 +2370,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.14" +version = "1.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" +checksum = "80af6f9131f277a45a3fba6ce8e2258037bb0477a67e610d3c1fe046ab31de47" [[package]] name = "ryu" @@ -2434,14 +2434,14 @@ checksum = "7eb0b34b42edc17f6b7cac84a52a1c5f0e1bb2227e997ca9011ea3dd34e8610b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.57", + "syn 2.0.59", ] [[package]] name = "serde_json" -version = "1.0.115" +version = "1.0.116" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12dc5c46daa8e9fdf4f5e71b6cf9a53f2487da0e86e55808e2d35539666497dd" +checksum = "3e17db7126d17feb94eb3fad46bf1a96b034e8aacbc2e775fe81505f8b0b2813" dependencies = [ "itoa", "ryu", @@ -2545,7 +2545,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.57", + "syn 2.0.59", ] [[package]] @@ -2566,32 +2566,13 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" -[[package]] -name = "strum" -version = "0.25.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" - [[package]] name = "strum" version = "0.26.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5d8cec3501a5194c432b2b7976db6b7d10ec95c253208b45f83f7136aa985e29" dependencies = [ - "strum_macros 0.26.2", -] - -[[package]] -name = "strum_macros" -version = "0.25.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23dc1fa9ac9c169a78ba62f0b841814b7abae11bdd047b9c58f893439e309ea0" -dependencies = [ - "heck 0.4.1", - "proc-macro2", - "quote", - "rustversion", - "syn 2.0.57", + "strum_macros", ] [[package]] @@ -2604,7 +2585,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.57", + "syn 2.0.59", ] [[package]] @@ -2649,9 +2630,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.57" +version = "2.0.59" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11a6ae1e52eb25aab8f3fb9fca13be982a373b8f1157ca14b897a825ba4a2d35" +checksum = "4a6531ffc7b071655e4ce2e04bd464c4830bb585a61cabb96cf808f05172615a" dependencies = [ "proc-macro2", "quote", @@ -2687,7 +2668,7 @@ checksum = "c61f3ba182994efc43764a46c018c347bc492c79f024e705f46567b418f6d4f7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.57", + "syn 2.0.59", ] [[package]] @@ -2790,7 +2771,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.57", + "syn 2.0.59", ] [[package]] @@ -2823,7 +2804,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.57", + "syn 2.0.59", ] [[package]] @@ -2971,7 +2952,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.57", + "syn 2.0.59", "wasm-bindgen-shared", ] @@ -2993,7 +2974,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.57", + "syn 2.0.59", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3063,7 +3044,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" dependencies = [ - "windows-targets 0.52.4", + "windows-targets 0.52.5", ] [[package]] @@ -3081,7 +3062,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" dependencies = [ - "windows-targets 0.52.4", + "windows-targets 0.52.5", ] [[package]] @@ -3116,17 +3097,18 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7dd37b7e5ab9018759f893a1952c9420d060016fc19a472b4bb20d1bdd694d1b" +checksum = "6f0713a46559409d202e70e28227288446bf7841d3211583a4b53e3f6d96e7eb" dependencies = [ - "windows_aarch64_gnullvm 0.52.4", - "windows_aarch64_msvc 0.52.4", - "windows_i686_gnu 0.52.4", - "windows_i686_msvc 0.52.4", - "windows_x86_64_gnu 0.52.4", - "windows_x86_64_gnullvm 0.52.4", - "windows_x86_64_msvc 0.52.4", + "windows_aarch64_gnullvm 0.52.5", + "windows_aarch64_msvc 0.52.5", + "windows_i686_gnu 0.52.5", + "windows_i686_gnullvm", + "windows_i686_msvc 0.52.5", + "windows_x86_64_gnu 0.52.5", + "windows_x86_64_gnullvm 0.52.5", + "windows_x86_64_msvc 0.52.5", ] [[package]] @@ -3143,9 +3125,9 @@ checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" [[package]] name = "windows_aarch64_gnullvm" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bcf46cf4c365c6f2d1cc93ce535f2c8b244591df96ceee75d8e83deb70a9cac9" +checksum = "7088eed71e8b8dda258ecc8bac5fb1153c5cffaf2578fc8ff5d61e23578d3263" [[package]] name = "windows_aarch64_msvc" @@ -3161,9 +3143,9 @@ checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" [[package]] name = "windows_aarch64_msvc" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da9f259dd3bcf6990b55bffd094c4f7235817ba4ceebde8e6d11cd0c5633b675" +checksum = "9985fd1504e250c615ca5f281c3f7a6da76213ebd5ccc9561496568a2752afb6" [[package]] name = "windows_i686_gnu" @@ -3179,9 +3161,15 @@ checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" [[package]] name = "windows_i686_gnu" -version = "0.52.4" +version = "0.52.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "88ba073cf16d5372720ec942a8ccbf61626074c6d4dd2e745299726ce8b89670" + +[[package]] +name = "windows_i686_gnullvm" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b474d8268f99e0995f25b9f095bc7434632601028cf86590aea5c8a5cb7801d3" +checksum = "87f4261229030a858f36b459e748ae97545d6f1ec60e5e0d6a3d32e0dc232ee9" [[package]] name = "windows_i686_msvc" @@ -3197,9 +3185,9 @@ checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" [[package]] name = "windows_i686_msvc" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1515e9a29e5bed743cb4415a9ecf5dfca648ce85ee42e15873c3cd8610ff8e02" +checksum = "db3c2bf3d13d5b658be73463284eaf12830ac9a26a90c717b7f771dfe97487bf" [[package]] name = "windows_x86_64_gnu" @@ -3215,9 +3203,9 @@ checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" [[package]] name = "windows_x86_64_gnu" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5eee091590e89cc02ad514ffe3ead9eb6b660aedca2183455434b93546371a03" +checksum = "4e4246f76bdeff09eb48875a0fd3e2af6aada79d409d33011886d3e1581517d9" [[package]] name = "windows_x86_64_gnullvm" @@ -3233,9 +3221,9 @@ checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" [[package]] name = "windows_x86_64_gnullvm" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77ca79f2451b49fa9e2af39f0747fe999fcda4f5e241b2898624dca97a1f2177" +checksum = "852298e482cd67c356ddd9570386e2862b5673c85bd5f88df9ab6802b334c596" [[package]] name = "windows_x86_64_msvc" @@ -3251,9 +3239,9 @@ checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" [[package]] name = "windows_x86_64_msvc" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32b752e52a2da0ddfbdbcc6fceadfeede4c939ed16d13e648833a61dfb611ed8" +checksum = "bec47e5bfd1bff0eeaf6d8b485cc1074891a197ab4225d504cb7a1ab88b02bf0" [[package]] name = "zerocopy" @@ -3272,7 +3260,7 @@ checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.57", + "syn 2.0.59", ] [[package]] From 1fbeda2bea525476eafa12b8d43def5364bf5977 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 15 Apr 2024 22:39:16 -0700 Subject: [PATCH 10/30] Update diff --- dev/diffs/3.4.2.diff | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 79609f65ee..4b39f2290c 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -210,6 +210,26 @@ index 0efe0877e9b..423d3b3d76d 100644 -- -- SELECT_HAVING -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +index ea5e47ede55..cab59443c79 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +@@ -24,6 +24,7 @@ import scala.util.Random + import org.scalatest.matchers.must.Matchers.the + + import org.apache.spark.SparkException ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec + import org.apache.spark.sql.execution.WholeStageCodegenExec + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +@@ -756,6 +757,7 @@ class DataFrameAggregateSuite extends QueryTest + + val exchangePlans = collect(aggPlan) { + case shuffle: ShuffleExchangeExec => shuffle ++ case shuffle: CometShuffleExchangeExec => shuffle + } + assert(exchangePlans.length == 1) + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 56e9520fdab..917932336df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala From 7e2c2c0104dda24e276e33d6d52cac34b058214a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Apr 2024 00:21:06 -0700 Subject: [PATCH 11/30] Update diff --- dev/diffs/3.4.2.diff | 38 ++++++++++++++++++++++++++++++++++++-- 1 file changed, 36 insertions(+), 2 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 4b39f2290c..0542b9baa3 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -246,10 +246,44 @@ index 56e9520fdab..917932336df 100644 spark.range(100).write.saveAsTable(s"$dbName.$table2Name") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala -index 9ddb4abe98b..1bebe99f1cc 100644 +index 9ddb4abe98b..2bd28d4041d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala -@@ -3311,7 +3311,8 @@ class DataFrameSuite extends QueryTest +@@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Attri + import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation + import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LeafNode, LocalRelation, LogicalPlan, OneRowRelation, Statistics} + import org.apache.spark.sql.catalyst.util.DateTimeUtils ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec + import org.apache.spark.sql.connector.FakeV2Provider + import org.apache.spark.sql.execution.{FilterExec, LogicalRDD, QueryExecution, SortExec, WholeStageCodegenExec} + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +@@ -2291,7 +2292,7 @@ class DataFrameSuite extends QueryTest + checkAnswer(join, df) + assert( + collect(join.queryExecution.executedPlan) { +- case e: ShuffleExchangeExec => true }.size === 1) ++ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => true }.size === 1) + assert( + collect(join.queryExecution.executedPlan) { case e: ReusedExchangeExec => true }.size === 1) + val broadcasted = broadcast(join) +@@ -2299,7 +2300,7 @@ class DataFrameSuite extends QueryTest + checkAnswer(join2, df) + assert( + collect(join2.queryExecution.executedPlan) { +- case e: ShuffleExchangeExec => true }.size == 1) ++ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => true }.size == 1) + assert( + collect(join2.queryExecution.executedPlan) { + case e: BroadcastExchangeExec => true }.size === 1) +@@ -2863,6 +2864,7 @@ class DataFrameSuite extends QueryTest + // Assert that no extra shuffle introduced by cogroup. + val exchanges = collect(df3.queryExecution.executedPlan) { + case h: ShuffleExchangeExec => h ++ case h: CometShuffleExchangeExec => h + } + assert(exchanges.size == 2) + } +@@ -3311,7 +3313,8 @@ class DataFrameSuite extends QueryTest assert(df2.isLocal) } From dd196c8fb451a008ac658b831419bbc9dcf6bced Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Apr 2024 23:55:22 -0700 Subject: [PATCH 12/30] Update diff --- dev/diffs/3.4.2.diff | 116 +++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 112 insertions(+), 4 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 0542b9baa3..f14185f696 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -453,18 +453,116 @@ index 00000000000..4b31bea33de + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -index 5125708be32..a1f1ae90796 100644 +index 5125708be32..123f58c522a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -@@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier +@@ -29,6 +29,8 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, GenericRow, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.comet._ ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} -@@ -1369,9 +1370,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -739,7 +741,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + } + } + +- test("test SortMergeJoin (with spill)") { ++ test("test SortMergeJoin (with spill)", ++ IgnoreComet("TODO: Comet SMJ doesn't support spill yet")) { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "0", + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> "1") { +@@ -868,14 +871,14 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + } + val executed = df.queryExecution.executedPlan + val executedJoins = collect(executed) { +- case j: SortMergeJoinExec => j ++ case j: CometSortMergeJoinExec => j + } + // This only applies to the above tested queries, in which a child SortMergeJoin always + // contains the SortOrder required by its parent SortMergeJoin. Thus, SortExec should never + // appear as parent of SortMergeJoin. + executed.foreach { + case s: SortExec => s.foreach { +- case j: SortMergeJoinExec => fail( ++ case j: CometSortMergeJoinExec => fail( + s"No extra sort should be added since $j already satisfies the required ordering" + ) + case _ => +@@ -1114,9 +1117,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val plan = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) + .groupBy($"k1").count() + .queryExecution.executedPlan +- assert(collect(plan) { case _: ShuffledHashJoinExec => true }.size === 1) ++ assert(collect(plan) { ++ case _: ShuffledHashJoinExec | _: CometHashJoinExec => true }.size === 1) + // No extra shuffle before aggregate +- assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 2) ++ assert(collect(plan) { ++ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => true }.size === 2) + }) + } + +@@ -1133,10 +1138,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) + .queryExecution + .executedPlan +- assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 2) ++ assert(collect(plan) { ++ case _: SortMergeJoinExec | _: CometSortMergeJoinExec => true }.size === 2) + assert(collect(plan) { case _: BroadcastHashJoinExec => true }.size === 1) + // No extra sort before last sort merge join +- assert(collect(plan) { case _: SortExec => true }.size === 3) ++ assert(collect(plan) { case _: SortExec | _: CometSortExec => true }.size === 3) + }) + + // Test shuffled hash join +@@ -1146,10 +1152,13 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) + .queryExecution + .executedPlan +- assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 2) +- assert(collect(plan) { case _: ShuffledHashJoinExec => true }.size === 1) ++ assert(collect(plan) { ++ case _: SortMergeJoinExec | _: CometSortMergeJoinExec => true }.size === 2) ++ assert(collect(plan) { ++ case _: ShuffledHashJoinExec | _: CometHashJoinExec => true }.size === 1) + // No extra sort before last sort merge join +- assert(collect(plan) { case _: SortExec => true }.size === 3) ++ assert(collect(plan) { ++ case _: SortExec | _: CometSortExec => true }.size === 3) + }) + } + +@@ -1240,12 +1249,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + inputDFs.foreach { case (df1, df2, joinExprs) => + val smjDF = df1.join(df2.hint("SHUFFLE_MERGE"), joinExprs, "full") + assert(collect(smjDF.queryExecution.executedPlan) { +- case _: SortMergeJoinExec => true }.size === 1) ++ case _: SortMergeJoinExec | _: CometSortMergeJoinExec => true }.size === 1) + val smjResult = smjDF.collect() + + val shjDF = df1.join(df2.hint("SHUFFLE_HASH"), joinExprs, "full") + assert(collect(shjDF.queryExecution.executedPlan) { +- case _: ShuffledHashJoinExec => true }.size === 1) ++ case _: ShuffledHashJoinExec | _: CometHashJoinExec => true }.size === 1) + // Same result between shuffled hash join and sort merge join + checkAnswer(shjDF, smjResult) + } +@@ -1340,7 +1349,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan + assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) + // Have shuffle before aggregation +- assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 1) ++ assert(collect(plan) { ++ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => true }.size === 1) + } + + def getJoinQuery(selectExpr: String, joinType: String): String = { +@@ -1369,9 +1379,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -479,7 +577,7 @@ index 5125708be32..a1f1ae90796 100644 } // Test output ordering is not preserved -@@ -1380,9 +1384,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1380,9 +1393,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val selectExpr = "/*+ BROADCAST(left_t) */ k1 as k0" val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -494,6 +592,16 @@ index 5125708be32..a1f1ae90796 100644 } // Test singe partition +@@ -1392,7 +1408,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + |FROM range(0, 10, 1, 1) t1 FULL OUTER JOIN range(0, 10, 1, 1) t2 + |""".stripMargin) + val plan = fullJoinDF.queryExecution.executedPlan +- assert(collect(plan) { case _: ShuffleExchangeExec => true}.size == 1) ++ assert(collect(plan) { ++ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => true}.size == 1) + checkAnswer(fullJoinDF, Row(100)) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index b5b34922694..a72403780c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala From 2b0bc0ba30d34661c8a4e46c74110f08c855592d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 20 Apr 2024 19:25:11 -0700 Subject: [PATCH 13/30] Update diff --- dev/diffs/3.4.2.diff | 66 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 66 insertions(+) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index f14185f696..e7f6310aaf 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -210,6 +210,28 @@ index 0efe0877e9b..423d3b3d76d 100644 -- -- SELECT_HAVING -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +index cf40e944c09..fc940f9452f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +@@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException + import org.apache.spark.sql.catalyst.expressions.SubqueryExpression + import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, Join, JoinStrategyHint, SHUFFLE_HASH} + import org.apache.spark.sql.catalyst.util.DateTimeConstants ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec + import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, RDDScanExec, SparkPlan} + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.columnar._ +@@ -516,7 +517,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils + */ + private def verifyNumExchanges(df: DataFrame, expected: Int): Unit = { + assert( +- collect(df.queryExecution.executedPlan) { case e: ShuffleExchangeExec => e }.size == expected) ++ collect(df.queryExecution.executedPlan) { ++ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => 1 }.size == expected) + } + + test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index ea5e47ede55..cab59443c79 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -829,6 +851,50 @@ index bd9c79e5b96..ab7584e768e 100644 } assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala +index ce43edb79c1..89e05c75380 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala +@@ -17,9 +17,10 @@ + + package org.apache.spark.sql.execution.datasources + +-import org.apache.spark.sql.{QueryTest, Row} ++import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, NullsFirst, SortOrder} + import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Sort} ++import org.apache.spark.sql.comet.CometSortExec + import org.apache.spark.sql.execution.{QueryExecution, SortExec} + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec + import org.apache.spark.sql.internal.SQLConf +@@ -224,7 +225,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + + // assert the outer most sort in the executed plan + assert(plan.collectFirst { +- case s: SortExec => s ++ case s: CometSortExec => s.originalPlan + }.exists { + case SortExec(Seq( + SortOrder(AttributeReference("key", IntegerType, _, _), Ascending, NullsFirst, _), +@@ -271,7 +272,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + + // assert the outer most sort in the executed plan + assert(plan.collectFirst { +- case s: SortExec => s ++ case s: CometSortExec => s.originalPlan + }.exists { + case SortExec(Seq( + SortOrder(AttributeReference("value", StringType, _, _), Ascending, NullsFirst, _), +@@ -305,7 +306,8 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + } + } + +- test("v1 write with AQE changing SMJ to BHJ") { ++ test("v1 write with AQE changing SMJ to BHJ", ++ IgnoreComet("TODO: Comet SMJ to BHJ by AQE")) { + withPlannedWrite { enabled => + withTable("t") { + sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala index 1d2e467c94c..3ea82cd1a3f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala From 733991bf158094f086447477a7f44b6db741acd6 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 22 Apr 2024 13:35:45 -0700 Subject: [PATCH 14/30] Update diff --- dev/diffs/3.4.2.diff | 36 ++++++++++++++++++++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index e7f6310aaf..de5e2c00ff 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -1570,6 +1570,42 @@ index 2a2a83d35e1..e3b7b290b3e 100644 val initialStateDS = Seq(("keyInStateAndData", new RunningCount(1))).toDS() val initialState: KeyValueGroupedDataset[String, RunningCount] = initialStateDS.groupByKey(_._1).mapValues(_._2) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +index ef5b8a769fe..84fe1bfabc9 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +@@ -37,6 +37,7 @@ import org.apache.spark.sql._ + import org.apache.spark.sql.catalyst.plans.logical.{Range, RepartitionByExpression} + import org.apache.spark.sql.catalyst.streaming.{InternalOutputModes, StreamingRelationV2} + import org.apache.spark.sql.catalyst.util.DateTimeUtils ++import org.apache.spark.sql.comet.CometLocalLimitExec + import org.apache.spark.sql.execution.{LocalLimitExec, SimpleMode, SparkPlan} + import org.apache.spark.sql.execution.command.ExplainCommand + import org.apache.spark.sql.execution.streaming._ +@@ -1103,11 +1104,12 @@ class StreamSuite extends StreamTest { + val localLimits = execPlan.collect { + case l: LocalLimitExec => l + case l: StreamingLocalLimitExec => l ++ case l: CometLocalLimitExec => l + } + + require( + localLimits.size == 1, +- s"Cant verify local limit optimization with this plan:\n$execPlan") ++ s"Cant verify local limit optimization ${localLimits.size} with this plan:\n$execPlan") + + if (expectStreamingLimit) { + assert( +@@ -1115,7 +1117,8 @@ class StreamSuite extends StreamTest { + s"Local limit was not StreamingLocalLimitExec:\n$execPlan") + } else { + assert( +- localLimits.head.isInstanceOf[LocalLimitExec], ++ localLimits.head.isInstanceOf[LocalLimitExec] || ++ localLimits.head.isInstanceOf[CometLocalLimitExec], + s"Local limit was not LocalLimitExec:\n$execPlan") + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index abe606ad9c1..2d930b64cca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala From eb41802042e33f34be3c396ce49d9ad811a6c032 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 29 Apr 2024 19:31:15 -0700 Subject: [PATCH 15/30] Fix more tests --- dev/diffs/3.4.2.diff | 267 +++++++++++++++++- .../plans/AliasAwareOutputExpression.scala | 3 +- 2 files changed, 267 insertions(+), 3 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index de5e2c00ff..c7aee4a5b9 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -474,6 +474,34 @@ index 00000000000..4b31bea33de + } + } +} +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala +index 1792b4c32eb..1616e6f39bd 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala +@@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide + import org.apache.spark.sql.catalyst.plans.PlanTest + import org.apache.spark.sql.catalyst.plans.logical._ + import org.apache.spark.sql.catalyst.rules.RuleExecutor ++import org.apache.spark.sql.comet.{CometHashJoinExec, CometSortMergeJoinExec} + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.joins._ + import org.apache.spark.sql.internal.SQLConf +@@ -362,6 +363,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession with AdaptiveSparkP + val executedPlan = df.queryExecution.executedPlan + val shuffleHashJoins = collect(executedPlan) { + case s: ShuffledHashJoinExec => s ++ case c: CometHashJoinExec => c.originalPlan.asInstanceOf[ShuffledHashJoinExec] + } + assert(shuffleHashJoins.size == 1) + assert(shuffleHashJoins.head.buildSide == buildSide) +@@ -371,6 +373,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession with AdaptiveSparkP + val executedPlan = df.queryExecution.executedPlan + val shuffleMergeJoins = collect(executedPlan) { + case s: SortMergeJoinExec => s ++ case c: CometSortMergeJoinExec => c + } + assert(shuffleMergeJoins.size == 1) + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 5125708be32..123f58c522a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -794,11 +822,168 @@ index 9e9d717db3b..91a4f9a38d5 100644 assert(actual == expected) } } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala +index 30ce940b032..0d3f6c6c934 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala +@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution + + import org.apache.spark.sql.{DataFrame, QueryTest} + import org.apache.spark.sql.catalyst.plans.physical.{RangePartitioning, UnknownPartitioning} ++import org.apache.spark.sql.comet.CometSortExec + import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} + import org.apache.spark.sql.execution.joins.ShuffledJoin + import org.apache.spark.sql.internal.SQLConf +@@ -33,7 +34,7 @@ abstract class RemoveRedundantSortsSuiteBase + + private def checkNumSorts(df: DataFrame, count: Int): Unit = { + val plan = df.queryExecution.executedPlan +- assert(collectWithSubqueries(plan) { case s: SortExec => s }.length == count) ++ assert(collectWithSubqueries(plan) { case _: SortExec | _: CometSortExec => 1 }.length == count) + } + + private def checkSorts(query: String, enabledCount: Int, disabledCount: Int): Unit = { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala +index 47679ed7865..9ffbaecb98e 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala +@@ -18,6 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.sql.{DataFrame, QueryTest} ++import org.apache.spark.sql.comet.CometHashAggregateExec + import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} + import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} + import org.apache.spark.sql.internal.SQLConf +@@ -31,7 +32,7 @@ abstract class ReplaceHashWithSortAggSuiteBase + private def checkNumAggs(df: DataFrame, hashAggCount: Int, sortAggCount: Int): Unit = { + val plan = df.queryExecution.executedPlan + assert(collectWithSubqueries(plan) { +- case s @ (_: HashAggregateExec | _: ObjectHashAggregateExec) => s ++ case s @ (_: HashAggregateExec | _: ObjectHashAggregateExec | _: CometHashAggregateExec ) => s + }.length == hashAggCount) + assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -index ac710c32296..37746bd470d 100644 +index ac710c32296..88a5329e74e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -@@ -616,7 +616,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution + + import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} + import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} ++import org.apache.spark.sql.comet.{CometProjectExec, CometSortExec, CometSortMergeJoinExec} + import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite + import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} + import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +@@ -192,7 +193,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val joinUniqueDF = df1.join(df2.hint(hint), $"k1" === $"k2", "full_outer") + assert(joinUniqueDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true +- case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(_: CometSortMergeJoinExec))) ++ if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(joinUniqueDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4), + Row(null, 5), Row(null, 6), Row(null, 7), Row(null, 8), Row(null, 9))) +@@ -202,7 +204,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val joinNonUniqueDF = df1.join(df2.hint(hint), $"k1" === $"k2" % 3, "full_outer") + assert(joinNonUniqueDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true +- case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(_: CometSortMergeJoinExec))) ++ if hint == "SHUFFLE_MERGE" => true + }.size === 1) + checkAnswer(joinNonUniqueDF, Seq(Row(0, 0), Row(0, 3), Row(0, 6), Row(0, 9), Row(1, 1), + Row(1, 4), Row(1, 7), Row(2, 2), Row(2, 5), Row(2, 8), Row(3, null), Row(4, null))) +@@ -224,6 +227,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(twoJoinsDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 2) + checkAnswer(twoJoinsDF, + Seq(Row(0, 0, 0), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, null), Row(4, 4, null), +@@ -240,7 +244,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + // test one left outer sort merge join + val oneLeftOuterJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_outer") + assert(oneLeftOuterJoinDF.queryExecution.executedPlan.collect { +- case WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(_: CometSortMergeJoinExec))) => true + }.size === 1) + checkAnswer(oneLeftOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, null), + Row(5, null), Row(6, null), Row(7, null), Row(8, null), Row(9, null))) +@@ -248,7 +252,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + // test one right outer sort merge join + val oneRightOuterJoinDF = df2.join(df3.hint("SHUFFLE_MERGE"), $"k2" === $"k3", "right_outer") + assert(oneRightOuterJoinDF.queryExecution.executedPlan.collect { +- case WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(_: CometSortMergeJoinExec))) => true + }.size === 1) + checkAnswer(oneRightOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(null, 4), + Row(null, 5))) +@@ -258,6 +262,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "right_outer") + assert(twoJoinsDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case _: CometSortMergeJoinExec => true + }.size === 2) + checkAnswer(twoJoinsDF, + Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, null, 4), Row(5, null, 5), +@@ -272,7 +277,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + // test one left semi sort merge join + val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_semi") + assert(oneJoinDF.queryExecution.executedPlan.collect { +- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true ++ case WholeStageCodegenExec( ++ ColumnarToRowExec(InputAdapter( ++ CometProjectExec(_, _, _, _, _: CometSortMergeJoinExec, _)))) => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(0), Row(1), Row(2), Row(3))) + +@@ -280,8 +287,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") + .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") + assert(twoJoinsDF.queryExecution.executedPlan.collect { +- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | +- WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case _: CometSortMergeJoinExec => true + }.size === 2) + checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) + } +@@ -294,7 +300,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + // test one left anti sort merge join + val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") + assert(oneJoinDF.queryExecution.executedPlan.collect { +- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true ++ case WholeStageCodegenExec( ++ ColumnarToRowExec(InputAdapter( ++ CometProjectExec(_, _, _, _, _: CometSortMergeJoinExec, _)))) => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(4), Row(5), Row(6), Row(7), Row(8), Row(9))) + +@@ -302,8 +310,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val twoJoinsDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") + .join(df3.hint("SHUFFLE_MERGE"), $"k1" === $"k3", "left_anti") + assert(twoJoinsDF.queryExecution.executedPlan.collect { +- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | +- WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case _: CometSortMergeJoinExec => true + }.size === 2) + checkAnswer(twoJoinsDF, Seq(Row(6), Row(7), Row(8), Row(9))) + } +@@ -436,7 +443,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val plan = df.queryExecution.executedPlan + assert(plan.exists(p => + p.isInstanceOf[WholeStageCodegenExec] && +- p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec])) ++ p.asInstanceOf[WholeStageCodegenExec].collect { ++ case _: CometSortExec => true ++ }.nonEmpty)) + assert(df.collect() === Array(Row(1), Row(2), Row(3))) + } + +@@ -616,7 +625,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .write.mode(SaveMode.Overwrite).parquet(path) withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "255", @@ -1606,6 +1791,84 @@ index ef5b8a769fe..84fe1bfabc9 100644 s"Local limit was not LocalLimitExec:\n$execPlan") } } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala +index b4c4ec7acbf..20579284856 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala +@@ -23,6 +23,7 @@ import org.apache.commons.io.FileUtils + import org.scalatest.Assertions + + import org.apache.spark.sql.catalyst.plans.physical.UnspecifiedDistribution ++import org.apache.spark.sql.comet.CometHashAggregateExec + import org.apache.spark.sql.execution.aggregate.BaseAggregateExec + import org.apache.spark.sql.execution.streaming.{MemoryStream, StateStoreRestoreExec, StateStoreSaveExec} + import org.apache.spark.sql.functions.count +@@ -67,6 +68,7 @@ class StreamingAggregationDistributionSuite extends StreamTest + // verify aggregations in between, except partial aggregation + val allAggregateExecs = query.lastExecution.executedPlan.collect { + case a: BaseAggregateExec => a ++ case c: CometHashAggregateExec => c.originalPlan + } + + val aggregateExecsWithoutPartialAgg = allAggregateExecs.filter { +@@ -201,6 +203,7 @@ class StreamingAggregationDistributionSuite extends StreamTest + // verify aggregations in between, except partial aggregation + val allAggregateExecs = executedPlan.collect { + case a: BaseAggregateExec => a ++ case c: CometHashAggregateExec => c.originalPlan + } + + val aggregateExecsWithoutPartialAgg = allAggregateExecs.filter { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +index 4d92e270539..33f1c2eb75e 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +@@ -31,7 +31,7 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation + import org.apache.spark.sql.{DataFrame, Row, SparkSession} + import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} + import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike + import org.apache.spark.sql.execution.streaming.{MemoryStream, StatefulOperatorStateInfo, StreamingSymmetricHashJoinExec, StreamingSymmetricHashJoinHelper} + import org.apache.spark.sql.execution.streaming.state.{RocksDBStateStoreProvider, StateStore, StateStoreProviderId} + import org.apache.spark.sql.functions._ +@@ -619,14 +619,28 @@ class StreamingInnerJoinSuite extends StreamingJoinSuite { + + val numPartitions = spark.sqlContext.conf.getConf(SQLConf.SHUFFLE_PARTITIONS) + +- assert(query.lastExecution.executedPlan.collect { +- case j @ StreamingSymmetricHashJoinExec(_, _, _, _, _, _, _, _, _, +- ShuffleExchangeExec(opA: HashPartitioning, _, _), +- ShuffleExchangeExec(opB: HashPartitioning, _, _)) +- if partitionExpressionsColumns(opA.expressions) === Seq("a", "b") +- && partitionExpressionsColumns(opB.expressions) === Seq("a", "b") +- && opA.numPartitions == numPartitions && opB.numPartitions == numPartitions => j +- }.size == 1) ++ val join = query.lastExecution.executedPlan.collect { ++ case j: StreamingSymmetricHashJoinExec => j ++ }.head ++ val opA = join.left.collect { ++ case s: ShuffleExchangeLike ++ if s.outputPartitioning.isInstanceOf[HashPartitioning] && ++ partitionExpressionsColumns( ++ s.outputPartitioning ++ .asInstanceOf[HashPartitioning].expressions) === Seq("a", "b") => ++ s.outputPartitioning ++ .asInstanceOf[HashPartitioning] ++ }.head ++ val opB = join.right.collect { ++ case s: ShuffleExchangeLike ++ if s.outputPartitioning.isInstanceOf[HashPartitioning] && ++ partitionExpressionsColumns( ++ s.outputPartitioning ++ .asInstanceOf[HashPartitioning].expressions) === Seq("a", "b") => ++ s.outputPartitioning ++ .asInstanceOf[HashPartitioning] ++ }.head ++ assert(opA.numPartitions == numPartitions && opB.numPartitions == numPartitions) + }) + } + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index abe606ad9c1..2d930b64cca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/plans/AliasAwareOutputExpression.scala b/spark/src/main/scala/org/apache/spark/sql/comet/plans/AliasAwareOutputExpression.scala index 6e5b44c8fa..65db2c9cf5 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/plans/AliasAwareOutputExpression.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/plans/AliasAwareOutputExpression.scala @@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.trees.CurrentOrigin trait AliasAwareOutputExpression extends SQLConfHelper { // `SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT` is Spark 3.4+ only. // Use a default value for now. - protected val aliasCandidateLimit = 100 + protected val aliasCandidateLimit = + conf.getConfString("spark.sql.optimizer.expressionProjectionCandidateLimit", "100").toInt protected def outputExpressions: Seq[NamedExpression] /** From 6b2315d62434966b9ef112951163d6f3161dbd27 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 30 Apr 2024 07:53:32 -0700 Subject: [PATCH 16/30] Fix more --- dev/diffs/3.4.2.diff | 93 ++++++++++++++++++++++++++++++++++++++++---- 1 file changed, 86 insertions(+), 7 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index c7aee4a5b9..3ed6f94cc0 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -315,6 +315,26 @@ index 9ddb4abe98b..2bd28d4041d 100644 withTable("tbl") { sql( """ +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +index 7dec558f8df..064cf6d4d97 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +@@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} + import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema + import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} + import org.apache.spark.sql.catalyst.util.sideBySide ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec + import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec, SQLExecution} + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} +@@ -2255,6 +2256,7 @@ class DatasetSuite extends QueryTest + // Assert that no extra shuffle introduced by cogroup. + val exchanges = collect(df3.queryExecution.executedPlan) { + case h: ShuffleExchangeExec => h ++ case c: CometShuffleExchangeExec => c + } + assert(exchanges.size == 2) + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index f33432ddb6f..060f874ea72 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -390,18 +410,26 @@ index a6b295578d6..a5cb616945a 100644 Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala -index 2796b1cf154..94591f83c84 100644 +index 2796b1cf154..be7078b38f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.TestingUDT.{IntervalUDT, NullData, NullUDT} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GreaterThan, Literal} import org.apache.spark.sql.catalyst.expressions.IntegralLiteralTestUtils.{negativeInt, positiveInt} import org.apache.spark.sql.catalyst.plans.logical.Filter -+import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec} ++import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.{FileSourceScanLike, SimpleMode} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.FilePartition -@@ -875,6 +876,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -815,6 +816,7 @@ class FileBasedDataSourceSuite extends QueryTest + assert(bJoinExec.isEmpty) + val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case smJoin: SortMergeJoinExec => smJoin ++ case smJoin: CometSortMergeJoinExec => smJoin + } + assert(smJoinExec.nonEmpty) + } +@@ -875,6 +877,7 @@ class FileBasedDataSourceSuite extends QueryTest val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _) => f @@ -409,7 +437,7 @@ index 2796b1cf154..94591f83c84 100644 } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.nonEmpty) -@@ -916,6 +918,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -916,6 +919,7 @@ class FileBasedDataSourceSuite extends QueryTest val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _) => f @@ -417,7 +445,7 @@ index 2796b1cf154..94591f83c84 100644 } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.isEmpty) -@@ -1100,6 +1103,8 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1100,6 +1104,8 @@ class FileBasedDataSourceSuite extends QueryTest val filters = df.queryExecution.executedPlan.collect { case f: FileSourceScanLike => f.dataFilters case b: BatchScanExec => b.scan.asInstanceOf[FileScan].dataFilters @@ -738,6 +766,45 @@ index cfc8b2cc845..c6fcfd7bd08 100644 } } finally { spark.listenerManager.unregister(listener) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +index cf76f6ca32c..8a7c2b894ad 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +@@ -22,6 +22,8 @@ import org.apache.spark.sql.{DataFrame, Row} + import org.apache.spark.sql.catalyst.InternalRow + import org.apache.spark.sql.catalyst.expressions.{Literal, TransformExpression} + import org.apache.spark.sql.catalyst.plans.physical ++import org.apache.spark.sql.comet.CometSortMergeJoinExec ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec + import org.apache.spark.sql.connector.catalog.Identifier + import org.apache.spark.sql.connector.catalog.InMemoryTableCatalog + import org.apache.spark.sql.connector.catalog.functions._ +@@ -31,7 +33,7 @@ import org.apache.spark.sql.connector.expressions.Expressions._ + import org.apache.spark.sql.execution.SparkPlan + import org.apache.spark.sql.execution.datasources.v2.BatchScanExec + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.joins.SortMergeJoinExec + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.internal.SQLConf._ +@@ -279,13 +281,15 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { + Row("bbb", 20, 250.0), Row("bbb", 20, 350.0), Row("ccc", 30, 400.50))) + } + +- private def collectShuffles(plan: SparkPlan): Seq[ShuffleExchangeExec] = { ++ private def collectShuffles(plan: SparkPlan): Seq[ShuffleExchangeLike] = { + // here we skip collecting shuffle operators that are not associated with SMJ + collect(plan) { + case s: SortMergeJoinExec => s ++ case c: CometSortMergeJoinExec => c.originalPlan + }.flatMap(smj => + collect(smj) { + case s: ShuffleExchangeExec => s ++ case c: CometShuffleExchangeExec => c + }) + } + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index c0ec8a58bd5..4e8bc6ed3c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -1369,10 +1436,22 @@ index 3a0bd35cb70..b28f06a757f 100644 val workDirPath = workDir.getAbsolutePath val input = spark.range(5).toDF("id") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala -index 26e61c6b58d..cde10983c68 100644 +index 26e61c6b58d..cb09d7e116a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala -@@ -737,7 +737,8 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils +@@ -45,8 +45,10 @@ import org.apache.spark.sql.util.QueryExecutionListener + import org.apache.spark.util.{AccumulatorContext, JsonProtocol} + + // Disable AQE because metric info is different with AQE on/off ++// This test suite runs tests against the metrics of physical operators. ++// Disabling it for Comet because the metrics are different with Comet enabled. + class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils +- with DisableAdaptiveExecutionSuite { ++ with DisableAdaptiveExecutionSuite with IgnoreCometSuite { + import testImplicits._ + + /** +@@ -737,7 +739,8 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils } } From 1059ef2efd6fc98d120555c3161e0fb44aa00239 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 30 Apr 2024 11:25:07 -0700 Subject: [PATCH 17/30] Fix --- docs/source/user-guide/configs.md | 2 +- .../spark/sql/comet/plans/AliasAwareOutputExpression.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 3a16cd47db..2a70033273 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -29,7 +29,7 @@ Comet provides the following configuration settings. | spark.comet.columnar.shuffle.async.enabled | Whether to enable asynchronous shuffle for Arrow-based shuffle. By default, this config is false. | false | | spark.comet.columnar.shuffle.async.max.thread.num | Maximum number of threads on an executor used for Comet async columnar shuffle. By default, this config is 100. This is the upper bound of total number of shuffle threads per executor. In other words, if the number of cores * the number of shuffle threads per task `spark.comet.columnar.shuffle.async.thread.num` is larger than this config. Comet will use this config as the number of shuffle threads per executor instead. | 100 | | spark.comet.columnar.shuffle.async.thread.num | Number of threads used for Comet async columnar shuffle per shuffle task. By default, this config is 3. Note that more threads means more memory requirement to buffer shuffle data before flushing to disk. Also, more threads may not always improve performance, and should be set based on the number of cores available. | 3 | -| spark.comet.columnar.shuffle.enabled | Force Comet to only use columnar shuffle for CometScan and Spark regular operators. If this is enabled, Comet native shuffle will not be enabled but only Arrow shuffle. By default, this config is false. | false | +| spark.comet.columnar.shuffle.enabled | Whether to enable Arrow-based columnar shuffle for Comet and Spark regular operators. If this is enabled, Comet prefers columnar shuffle than native shuffle. By default, this config is true. | true | | spark.comet.columnar.shuffle.memory.factor | Fraction of Comet memory to be allocated per executor process for Comet shuffle. Comet memory size is specified by `spark.comet.memoryOverhead` or calculated by `spark.comet.memory.overhead.factor` * `spark.executor.memory`. By default, this config is 1.0. | 1.0 | | spark.comet.debug.enabled | Whether to enable debug mode for Comet. By default, this config is false. When enabled, Comet will do additional checks for debugging purpose. For example, validating array when importing arrays from JVM at native side. Note that these checks may be expensive in performance and should only be enabled for debugging purpose. | false | | spark.comet.enabled | Whether to enable Comet extension for Spark. When this is turned on, Spark will use Comet to read Parquet data source. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. By default, this config is the value of the env var `ENABLE_COMET` if set, or true otherwise. | true | diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/plans/AliasAwareOutputExpression.scala b/spark/src/main/scala/org/apache/spark/sql/comet/plans/AliasAwareOutputExpression.scala index 65db2c9cf5..996526e55e 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/plans/AliasAwareOutputExpression.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/plans/AliasAwareOutputExpression.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.trees.CurrentOrigin trait AliasAwareOutputExpression extends SQLConfHelper { // `SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT` is Spark 3.4+ only. // Use a default value for now. - protected val aliasCandidateLimit = + protected val aliasCandidateLimit: Int = conf.getConfString("spark.sql.optimizer.expressionProjectionCandidateLimit", "100").toInt protected def outputExpressions: Seq[NamedExpression] From 5cb2c5410ad7e52384d1986e3ceac0f608e14615 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 30 Apr 2024 14:02:41 -0700 Subject: [PATCH 18/30] Fix more --- dev/diffs/3.4.2.diff | 102 ++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 101 insertions(+), 1 deletion(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 3ed6f94cc0..0cd680a591 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -396,7 +396,7 @@ index f33432ddb6f..060f874ea72 100644 } assert(scanOption.isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -index a6b295578d6..a5cb616945a 100644 +index a6b295578d6..91acca4306f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -463,7 +463,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite @@ -409,6 +409,17 @@ index a6b295578d6..a5cb616945a 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt +@@ -541,7 +542,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +-class ExplainSuiteAE extends ExplainSuiteHelper with EnableAdaptiveExecutionSuite { ++// Ignored when Comet is enabled. Comet changes expected query plans. ++class ExplainSuiteAE extends ExplainSuiteHelper with EnableAdaptiveExecutionSuite ++ with IgnoreCometSuite { + import testImplicits._ + + test("SPARK-35884: Explain Formatted") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 2796b1cf154..be7078b38f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -741,6 +752,58 @@ index 3cfda19134a..278bb1060c4 100644 } assert(exchanges.size === 1) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +index 02990a7a40d..2f89e7eb391 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +@@ -24,6 +24,8 @@ import test.org.apache.spark.sql.connector._ + + import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} + import org.apache.spark.sql.catalyst.InternalRow ++import org.apache.spark.sql.comet.CometSortExec ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec + import org.apache.spark.sql.connector.catalog.{PartitionInternalRow, SupportsRead, Table, TableCapability, TableProvider} + import org.apache.spark.sql.connector.catalog.TableCapability._ + import org.apache.spark.sql.connector.expressions.{Expression, FieldReference, Literal, NamedReference, NullOrdering, SortDirection, SortOrder, Transform} +@@ -269,12 +271,14 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS + checkAnswer(groupByColJ, Seq(Row(2, 8), Row(4, 2), Row(6, 5))) + assert(collectFirst(groupByColJ.queryExecution.executedPlan) { + case e: ShuffleExchangeExec => e ++ case c: CometShuffleExchangeExec => c + }.isDefined) + + val groupByIPlusJ = df.groupBy($"i" + $"j").agg(count("*")) + checkAnswer(groupByIPlusJ, Seq(Row(5, 2), Row(6, 2), Row(8, 1), Row(9, 1))) + assert(collectFirst(groupByIPlusJ.queryExecution.executedPlan) { + case e: ShuffleExchangeExec => e ++ case c: CometShuffleExchangeExec => c + }.isDefined) + } + } +@@ -335,9 +339,11 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS + val (shuffleExpected, sortExpected) = groupByExpects + assert(collectFirst(groupBy.queryExecution.executedPlan) { + case e: ShuffleExchangeExec => e ++ case c: CometShuffleExchangeExec => c + }.isDefined === shuffleExpected) + assert(collectFirst(groupBy.queryExecution.executedPlan) { + case e: SortExec => e ++ case c: CometSortExec => c + }.isDefined === sortExpected) + } + +@@ -353,9 +359,11 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS + val (shuffleExpected, sortExpected) = windowFuncExpects + assert(collectFirst(windowPartByColIOrderByColJ.queryExecution.executedPlan) { + case e: ShuffleExchangeExec => e ++ case c: CometShuffleExchangeExec => c + }.isDefined === shuffleExpected) + assert(collectFirst(windowPartByColIOrderByColJ.queryExecution.executedPlan) { + case e: SortExec => e ++ case c: CometSortExec => c + }.isDefined === sortExpected) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala index cfc8b2cc845..c6fcfd7bd08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala @@ -865,6 +928,30 @@ index 743ec41dbe7..9f30d6c8e04 100644 case _: LeafExecNode => true case _ => false } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +index 4b3d3a4b805..56e1e0e6f16 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.rdd.RDD +-import org.apache.spark.sql.{execution, DataFrame, Row} ++import org.apache.spark.sql.{execution, DataFrame, IgnoreCometSuite, Row} + import org.apache.spark.sql.catalyst.InternalRow + import org.apache.spark.sql.catalyst.expressions._ + import org.apache.spark.sql.catalyst.plans._ +@@ -35,7 +35,9 @@ import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.sql.types._ + +-class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { ++// Ignore this suite when Comet is enabled. This suite tests the Spark planner and Comet planner ++// comes out with too many difference. Simply ignoring this suite for now. ++class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper with IgnoreCometSuite { + import testImplicits._ + + setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala index 9e9d717db3b..91a4f9a38d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -2056,6 +2143,19 @@ index ed2e309fa07..f51a2395a53 100644 conf.set( StaticSQLConf.WAREHOUSE_PATH, conf.get(StaticSQLConf.WAREHOUSE_PATH) + "/" + getClass.getCanonicalName) +diff --git a/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala +index 1510e8957f9..7618419d8ff 100644 +--- a/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala +@@ -43,7 +43,7 @@ class SqlResourceWithActualMetricsSuite + import testImplicits._ + + // Exclude nodes which may not have the metrics +- val excludedNodes = List("WholeStageCodegen", "Project", "SerializeFromObject") ++ val excludedNodes = List("WholeStageCodegen", "Project", "SerializeFromObject", "RowToColumnar") + + implicit val formats = new DefaultFormats { + override def dateFormatter = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala index 52abd248f3a..7a199931a08 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala From b342b66b71378de5ff511437ac0da5a239291ae8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 1 May 2024 17:36:16 -0700 Subject: [PATCH 19/30] Fix more --- dev/diffs/3.4.2.diff | 87 ++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 84 insertions(+), 3 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 0cd680a591..cae4a5e39b 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -1149,18 +1149,27 @@ index ac710c32296..88a5329e74e 100644 val df = spark.read.parquet(path).selectExpr(projection: _*) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -index 593bd7bb4ba..be1b82d0030 100644 +index 593bd7bb4ba..43a9fa1469d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -@@ -29,6 +29,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListe +@@ -29,6 +29,8 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListe import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession, Strategy} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} +import org.apache.spark.sql.comet._ ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.{CollectLimitExec, LocalTableScanExec, PartialReducerPartitionSpec, QueryExecution, ReusedSubqueryExec, ShuffledRowRDD, SortExec, SparkPlan, SparkPlanInfo, UnionExec} import org.apache.spark.sql.execution.aggregate.BaseAggregateExec import org.apache.spark.sql.execution.command.DataWritingCommandExec -@@ -116,6 +117,9 @@ class AdaptiveQueryExecSuite +@@ -104,6 +106,7 @@ class AdaptiveQueryExecSuite + private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = { + collect(plan) { + case j: BroadcastHashJoinExec => j ++ case j: CometBroadcastHashJoinExec => j.originalPlan.asInstanceOf[BroadcastHashJoinExec] + } + } + +@@ -116,6 +119,9 @@ class AdaptiveQueryExecSuite private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { collect(plan) { case j: SortMergeJoinExec => j @@ -1170,6 +1179,78 @@ index 593bd7bb4ba..be1b82d0030 100644 } } +@@ -134,12 +140,14 @@ class AdaptiveQueryExecSuite + private def findTopLevelSort(plan: SparkPlan): Seq[SortExec] = { + collect(plan) { + case s: SortExec => s ++ case s: CometSortExec => s.originalPlan.asInstanceOf[SortExec] + } + } + + private def findTopLevelAggregate(plan: SparkPlan): Seq[BaseAggregateExec] = { + collect(plan) { + case agg: BaseAggregateExec => agg ++ case agg: CometHashAggregateExec => agg.originalPlan.asInstanceOf[BaseAggregateExec] + } + } + +@@ -176,6 +184,7 @@ class AdaptiveQueryExecSuite + val parts = rdd.partitions + assert(parts.forall(rdd.preferredLocations(_).nonEmpty)) + } ++ + assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) + } + +@@ -1600,6 +1609,7 @@ class AdaptiveQueryExecSuite + "SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") + assert(collect(adaptivePlan) { + case s: ShuffleExchangeExec => s ++ case c: CometShuffleExchangeExec => c + }.length == 1) + } + } +@@ -2269,6 +2279,7 @@ class AdaptiveQueryExecSuite + } + + withSQLConf( ++ "spark.comet.enabled" -> "false", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val query = "SELECT * FROM testData join testData2 ON key = a where value = '1'" +@@ -2404,6 +2415,7 @@ class AdaptiveQueryExecSuite + val (_, adaptive) = runAdaptiveAndVerifyResult(query) + assert(adaptive.collect { + case sort: SortExec => sort ++ case sort: CometSortExec => sort + }.size == 1) + val read = collect(adaptive) { + case read: AQEShuffleReadExec => read +@@ -2424,6 +2436,8 @@ class AdaptiveQueryExecSuite + test("SPARK-37357: Add small partition factor for rebalance partitions") { + withTempView("v") { + withSQLConf( ++ // Comet shuffle changes partition size which makes the test fail ++ "spark.comet.enabled" -> "false", + SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + spark.sparkContext.parallelize( +@@ -2534,6 +2548,7 @@ class AdaptiveQueryExecSuite + "JOIN skewData3 ON value2 = value3") + val shuffles1 = collect(adaptive1) { + case s: ShuffleExchangeExec => s ++ case c: CometShuffleExchangeExec => c + } + assert(shuffles1.size == 4) + val smj1 = findTopLevelSortMergeJoin(adaptive1) +@@ -2545,6 +2560,7 @@ class AdaptiveQueryExecSuite + "JOIN skewData3 ON value1 = value3") + val shuffles2 = collect(adaptive2) { + case s: ShuffleExchangeExec => s ++ case c: CometShuffleExchangeExec => c + } + assert(shuffles2.size == 4) + val smj2 = findTopLevelSortMergeJoin(adaptive2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index bd9c79e5b96..ab7584e768e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala From 6d27a5f0903334fe15853cec841cce1caf37f063 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 1 May 2024 23:15:10 -0700 Subject: [PATCH 20/30] Fix more --- dev/diffs/3.4.2.diff | 191 ++++++++++++++++++++++++++++++++++++++----- 1 file changed, 169 insertions(+), 22 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index cae4a5e39b..275f479ace 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -1149,11 +1149,15 @@ index ac710c32296..88a5329e74e 100644 val df = spark.read.parquet(path).selectExpr(projection: _*) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -index 593bd7bb4ba..43a9fa1469d 100644 +index 593bd7bb4ba..1b21f44cea3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -@@ -29,6 +29,8 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListe - import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession, Strategy} +@@ -26,9 +26,11 @@ import org.scalatest.time.SpanSugar._ + + import org.apache.spark.SparkException + import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart} +-import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession, Strategy} ++import org.apache.spark.sql.{Dataset, IgnoreComet, QueryTest, Row, SparkSession, Strategy} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} +import org.apache.spark.sql.comet._ @@ -1169,7 +1173,7 @@ index 593bd7bb4ba..43a9fa1469d 100644 } } -@@ -116,6 +119,9 @@ class AdaptiveQueryExecSuite +@@ -116,30 +119,38 @@ class AdaptiveQueryExecSuite private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { collect(plan) { case j: SortMergeJoinExec => j @@ -1179,7 +1183,21 @@ index 593bd7bb4ba..43a9fa1469d 100644 } } -@@ -134,12 +140,14 @@ class AdaptiveQueryExecSuite + private def findTopLevelShuffledHashJoin(plan: SparkPlan): Seq[ShuffledHashJoinExec] = { + collect(plan) { + case j: ShuffledHashJoinExec => j ++ case j: CometHashJoinExec => j.originalPlan.asInstanceOf[ShuffledHashJoinExec] + } + } + + private def findTopLevelBaseJoin(plan: SparkPlan): Seq[BaseJoinExec] = { + collect(plan) { + case j: BaseJoinExec => j ++ case c: CometHashJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] ++ case c: CometSortMergeJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] + } + } + private def findTopLevelSort(plan: SparkPlan): Seq[SortExec] = { collect(plan) { case s: SortExec => s @@ -1194,7 +1212,7 @@ index 593bd7bb4ba..43a9fa1469d 100644 } } -@@ -176,6 +184,7 @@ class AdaptiveQueryExecSuite +@@ -176,6 +187,7 @@ class AdaptiveQueryExecSuite val parts = rdd.partitions assert(parts.forall(rdd.preferredLocations(_).nonEmpty)) } @@ -1202,7 +1220,67 @@ index 593bd7bb4ba..43a9fa1469d 100644 assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) } -@@ -1600,6 +1609,7 @@ class AdaptiveQueryExecSuite +@@ -185,6 +197,7 @@ class AdaptiveQueryExecSuite + assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) + val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect { + case s: ShuffleExchangeExec => s ++ case c: CometShuffleExchangeExec => c + } + assert(shuffle.size == 1) + assert(shuffle(0).outputPartitioning.numPartitions == numPartition) +@@ -539,7 +552,9 @@ class AdaptiveQueryExecSuite + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) +- checkNumLocalShuffleReads(adaptivePlan) ++ // Comet shuffle changes shuffle metrics, ++ // so we can't check the number of local shuffle reads. ++ // checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) +@@ -560,7 +575,9 @@ class AdaptiveQueryExecSuite + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) +- checkNumLocalShuffleReads(adaptivePlan) ++ // Comet shuffle changes shuffle metrics, ++ // so we can't check the number of local shuffle reads. ++ // checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.isEmpty) +@@ -569,7 +586,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("Broadcast exchange reuse across subqueries") { ++ test("Broadcast exchange reuse across subqueries", ++ IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", +@@ -786,7 +804,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("SPARK-29544: adaptive skew join with different join types") { ++ test("SPARK-29544: adaptive skew join with different join types", ++ IgnoreComet("Comet shuffle has different partition metrics")) { + Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { joinHint => + def getJoinNode(plan: SparkPlan): Seq[ShuffledJoin] = if (joinHint == "SHUFFLE_MERGE") { + findTopLevelSortMergeJoin(plan) +@@ -1004,7 +1023,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("metrics of the shuffle read") { ++ test("metrics of the shuffle read", ++ IgnoreComet("Comet shuffle changes the metrics")) { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT key FROM testData GROUP BY key") +@@ -1600,6 +1620,7 @@ class AdaptiveQueryExecSuite "SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") assert(collect(adaptivePlan) { case s: ShuffleExchangeExec => s @@ -1210,15 +1288,83 @@ index 593bd7bb4ba..43a9fa1469d 100644 }.length == 1) } } -@@ -2269,6 +2279,7 @@ class AdaptiveQueryExecSuite +@@ -1679,7 +1700,8 @@ class AdaptiveQueryExecSuite } + } - withSQLConf( -+ "spark.comet.enabled" -> "false", - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { - val query = "SELECT * FROM testData join testData2 ON key = a where value = '1'" -@@ -2404,6 +2415,7 @@ class AdaptiveQueryExecSuite +- test("SPARK-33551: Do not use AQE shuffle read for repartition") { ++ test("SPARK-33551: Do not use AQE shuffle read for repartition", ++ IgnoreComet("Comet shuffle changes partition size")) { + def hasRepartitionShuffle(plan: SparkPlan): Boolean = { + find(plan) { + case s: ShuffleExchangeLike => +@@ -1864,6 +1886,9 @@ class AdaptiveQueryExecSuite + def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { + assert(collect(ds.queryExecution.executedPlan) { + case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s ++ case c: CometShuffleExchangeExec ++ if c.originalPlan.shuffleOrigin == origin && ++ c.originalPlan.numPartitions == 2 => c + }.size == 1) + ds.collect() + val plan = ds.queryExecution.executedPlan +@@ -1872,6 +1897,9 @@ class AdaptiveQueryExecSuite + }.isEmpty) + assert(collect(plan) { + case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s ++ case c: CometShuffleExchangeExec ++ if c.originalPlan.shuffleOrigin == origin && ++ c.originalPlan.numPartitions == 2 => c + }.size == 1) + checkAnswer(ds, testData) + } +@@ -2028,7 +2056,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("SPARK-35264: Support AQE side shuffled hash join formula") { ++ test("SPARK-35264: Support AQE side shuffled hash join formula", ++ IgnoreComet("Comet shuffle changes the partition size")) { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { + Seq("100", "100000").foreach { size => +@@ -2114,7 +2143,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("SPARK-35725: Support optimize skewed partitions in RebalancePartitions") { ++ test("SPARK-35725: Support optimize skewed partitions in RebalancePartitions", ++ IgnoreComet("Comet shuffle changes shuffle metrics")) { + withTempView("v") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", +@@ -2214,6 +2244,7 @@ class AdaptiveQueryExecSuite + s"JOIN skewData2 ON key1 = key2 GROUP BY key1") + val shuffles1 = collect(adaptive1) { + case s: ShuffleExchangeExec => s ++ case c: CometShuffleExchangeExec => c.originalPlan + } + assert(shuffles1.size == 3) + // shuffles1.head is the top-level shuffle under the Aggregate operator +@@ -2227,6 +2258,7 @@ class AdaptiveQueryExecSuite + s"JOIN skewData2 ON key1 = key2") + val shuffles2 = collect(adaptive2) { + case s: ShuffleExchangeExec => s ++ case c: CometShuffleExchangeExec => c.originalPlan + } + if (hasRequiredDistribution) { + assert(shuffles2.size == 3) +@@ -2260,7 +2292,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("SPARK-35794: Allow custom plugin for cost evaluator") { ++ test("SPARK-35794: Allow custom plugin for cost evaluator", ++ IgnoreComet("Comet shuffle changes shuffle metrics")) { + CostEvaluator.instantiate( + classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, spark.sparkContext.getConf) + intercept[IllegalArgumentException] { +@@ -2404,6 +2437,7 @@ class AdaptiveQueryExecSuite val (_, adaptive) = runAdaptiveAndVerifyResult(query) assert(adaptive.collect { case sort: SortExec => sort @@ -1226,16 +1372,17 @@ index 593bd7bb4ba..43a9fa1469d 100644 }.size == 1) val read = collect(adaptive) { case read: AQEShuffleReadExec => read -@@ -2424,6 +2436,8 @@ class AdaptiveQueryExecSuite - test("SPARK-37357: Add small partition factor for rebalance partitions") { +@@ -2421,7 +2455,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("SPARK-37357: Add small partition factor for rebalance partitions") { ++ test("SPARK-37357: Add small partition factor for rebalance partitions", ++ IgnoreComet("Comet shuffle changes shuffle metrics")) { withTempView("v") { withSQLConf( -+ // Comet shuffle changes partition size which makes the test fail -+ "spark.comet.enabled" -> "false", SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", - SQLConf.SHUFFLE_PARTITIONS.key -> "1") { - spark.sparkContext.parallelize( -@@ -2534,6 +2548,7 @@ class AdaptiveQueryExecSuite +@@ -2534,6 +2569,7 @@ class AdaptiveQueryExecSuite "JOIN skewData3 ON value2 = value3") val shuffles1 = collect(adaptive1) { case s: ShuffleExchangeExec => s @@ -1243,7 +1390,7 @@ index 593bd7bb4ba..43a9fa1469d 100644 } assert(shuffles1.size == 4) val smj1 = findTopLevelSortMergeJoin(adaptive1) -@@ -2545,6 +2560,7 @@ class AdaptiveQueryExecSuite +@@ -2545,6 +2581,7 @@ class AdaptiveQueryExecSuite "JOIN skewData3 ON value1 = value3") val shuffles2 = collect(adaptive2) { case s: ShuffleExchangeExec => s From 508cb1adc18696acedfee9c2d58081c8c4e081d0 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 2 May 2024 14:34:08 -0700 Subject: [PATCH 21/30] Fix more --- dev/diffs/3.4.2.diff | 121 ++++++++++++++++++++++++++++++++++++------- 1 file changed, 102 insertions(+), 19 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 275f479ace..eb63946728 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -1149,7 +1149,7 @@ index ac710c32296..88a5329e74e 100644 val df = spark.read.parquet(path).selectExpr(projection: _*) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -index 593bd7bb4ba..1b21f44cea3 100644 +index 593bd7bb4ba..c736e1fcc54 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -26,9 +26,11 @@ import org.scalatest.time.SpanSugar._ @@ -1228,7 +1228,90 @@ index 593bd7bb4ba..1b21f44cea3 100644 } assert(shuffle.size == 1) assert(shuffle(0).outputPartitioning.numPartitions == numPartition) -@@ -539,7 +552,9 @@ class AdaptiveQueryExecSuite +@@ -241,7 +254,8 @@ class AdaptiveQueryExecSuite + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } +- assert(localReads.length == 2) ++ // Comet shuffle changes shuffle metrics ++ assert(localReads.length == 1) + val localShuffleRDD0 = localReads(0).execute().asInstanceOf[ShuffledRowRDD] + val localShuffleRDD1 = localReads(1).execute().asInstanceOf[ShuffledRowRDD] + // The pre-shuffle partition size is [0, 0, 0, 72, 0] +@@ -273,7 +287,8 @@ class AdaptiveQueryExecSuite + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } +- assert(localReads.length == 2) ++ // Comet shuffle changes shuffle metrics ++ assert(localReads.length == 1) + val localShuffleRDD0 = localReads(0).execute().asInstanceOf[ShuffledRowRDD] + val localShuffleRDD1 = localReads(1).execute().asInstanceOf[ShuffledRowRDD] + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 +@@ -298,7 +313,7 @@ class AdaptiveQueryExecSuite + .groupBy($"a").count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan +- assert(find(plan)(_.isInstanceOf[SortMergeJoinExec]).isDefined) ++ assert(find(plan)(_.isInstanceOf[CometSortMergeJoinExec]).isDefined) + val coalescedReads = collect(plan) { + case r: AQEShuffleReadExec => r + } +@@ -322,7 +337,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("Scalar subquery") { ++ test("Scalar subquery", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -337,7 +352,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("Scalar subquery in later stages") { ++ test("Scalar subquery in later stages", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -353,7 +368,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("multiple joins") { ++ test("multiple joins", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -398,7 +413,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("multiple joins with aggregate") { ++ test("multiple joins with aggregate", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -443,7 +458,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("multiple joins with aggregate 2") { ++ test("multiple joins with aggregate 2", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { +@@ -508,7 +523,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("Exchange reuse with subqueries") { ++ test("Exchange reuse with subqueries", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -539,7 +554,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1239,7 +1322,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) -@@ -560,7 +575,9 @@ class AdaptiveQueryExecSuite +@@ -560,7 +577,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1250,7 +1333,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.isEmpty) -@@ -569,7 +586,8 @@ class AdaptiveQueryExecSuite +@@ -569,7 +588,8 @@ class AdaptiveQueryExecSuite } } @@ -1260,7 +1343,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", -@@ -786,7 +804,8 @@ class AdaptiveQueryExecSuite +@@ -786,7 +806,8 @@ class AdaptiveQueryExecSuite } } @@ -1270,7 +1353,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { joinHint => def getJoinNode(plan: SparkPlan): Seq[ShuffledJoin] = if (joinHint == "SHUFFLE_MERGE") { findTopLevelSortMergeJoin(plan) -@@ -1004,7 +1023,8 @@ class AdaptiveQueryExecSuite +@@ -1004,7 +1025,8 @@ class AdaptiveQueryExecSuite } } @@ -1280,7 +1363,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT key FROM testData GROUP BY key") -@@ -1600,6 +1620,7 @@ class AdaptiveQueryExecSuite +@@ -1600,6 +1622,7 @@ class AdaptiveQueryExecSuite "SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") assert(collect(adaptivePlan) { case s: ShuffleExchangeExec => s @@ -1288,7 +1371,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 }.length == 1) } } -@@ -1679,7 +1700,8 @@ class AdaptiveQueryExecSuite +@@ -1679,7 +1702,8 @@ class AdaptiveQueryExecSuite } } @@ -1298,7 +1381,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 def hasRepartitionShuffle(plan: SparkPlan): Boolean = { find(plan) { case s: ShuffleExchangeLike => -@@ -1864,6 +1886,9 @@ class AdaptiveQueryExecSuite +@@ -1864,6 +1888,9 @@ class AdaptiveQueryExecSuite def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { assert(collect(ds.queryExecution.executedPlan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1308,7 +1391,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 }.size == 1) ds.collect() val plan = ds.queryExecution.executedPlan -@@ -1872,6 +1897,9 @@ class AdaptiveQueryExecSuite +@@ -1872,6 +1899,9 @@ class AdaptiveQueryExecSuite }.isEmpty) assert(collect(plan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1318,7 +1401,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 }.size == 1) checkAnswer(ds, testData) } -@@ -2028,7 +2056,8 @@ class AdaptiveQueryExecSuite +@@ -2028,7 +2058,8 @@ class AdaptiveQueryExecSuite } } @@ -1328,7 +1411,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 withTempView("t1", "t2") { def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { Seq("100", "100000").foreach { size => -@@ -2114,7 +2143,8 @@ class AdaptiveQueryExecSuite +@@ -2114,7 +2145,8 @@ class AdaptiveQueryExecSuite } } @@ -1338,7 +1421,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", -@@ -2214,6 +2244,7 @@ class AdaptiveQueryExecSuite +@@ -2214,6 +2246,7 @@ class AdaptiveQueryExecSuite s"JOIN skewData2 ON key1 = key2 GROUP BY key1") val shuffles1 = collect(adaptive1) { case s: ShuffleExchangeExec => s @@ -1346,7 +1429,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 } assert(shuffles1.size == 3) // shuffles1.head is the top-level shuffle under the Aggregate operator -@@ -2227,6 +2258,7 @@ class AdaptiveQueryExecSuite +@@ -2227,6 +2260,7 @@ class AdaptiveQueryExecSuite s"JOIN skewData2 ON key1 = key2") val shuffles2 = collect(adaptive2) { case s: ShuffleExchangeExec => s @@ -1354,7 +1437,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 } if (hasRequiredDistribution) { assert(shuffles2.size == 3) -@@ -2260,7 +2292,8 @@ class AdaptiveQueryExecSuite +@@ -2260,7 +2294,8 @@ class AdaptiveQueryExecSuite } } @@ -1364,7 +1447,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 CostEvaluator.instantiate( classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, spark.sparkContext.getConf) intercept[IllegalArgumentException] { -@@ -2404,6 +2437,7 @@ class AdaptiveQueryExecSuite +@@ -2404,6 +2439,7 @@ class AdaptiveQueryExecSuite val (_, adaptive) = runAdaptiveAndVerifyResult(query) assert(adaptive.collect { case sort: SortExec => sort @@ -1372,7 +1455,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 }.size == 1) val read = collect(adaptive) { case read: AQEShuffleReadExec => read -@@ -2421,7 +2455,8 @@ class AdaptiveQueryExecSuite +@@ -2421,7 +2457,8 @@ class AdaptiveQueryExecSuite } } @@ -1382,7 +1465,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", -@@ -2534,6 +2569,7 @@ class AdaptiveQueryExecSuite +@@ -2534,6 +2571,7 @@ class AdaptiveQueryExecSuite "JOIN skewData3 ON value2 = value3") val shuffles1 = collect(adaptive1) { case s: ShuffleExchangeExec => s @@ -1390,7 +1473,7 @@ index 593bd7bb4ba..1b21f44cea3 100644 } assert(shuffles1.size == 4) val smj1 = findTopLevelSortMergeJoin(adaptive1) -@@ -2545,6 +2581,7 @@ class AdaptiveQueryExecSuite +@@ -2545,6 +2583,7 @@ class AdaptiveQueryExecSuite "JOIN skewData3 ON value1 = value3") val shuffles2 = collect(adaptive2) { case s: ShuffleExchangeExec => s From 5e03881af92bd6299fc66f3fcdc25b63c7fe705e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 2 May 2024 21:03:33 -0700 Subject: [PATCH 22/30] Fix more --- dev/diffs/3.4.2.diff | 102 ++++++++++++++++++++++++++++--------------- 1 file changed, 66 insertions(+), 36 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index eb63946728..eba47b690f 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -1149,7 +1149,7 @@ index ac710c32296..88a5329e74e 100644 val df = spark.read.parquet(path).selectExpr(projection: _*) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -index 593bd7bb4ba..c736e1fcc54 100644 +index 593bd7bb4ba..f39fe59f36b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -26,9 +26,11 @@ import org.scalatest.time.SpanSugar._ @@ -1228,17 +1228,37 @@ index 593bd7bb4ba..c736e1fcc54 100644 } assert(shuffle.size == 1) assert(shuffle(0).outputPartitioning.numPartitions == numPartition) -@@ -241,7 +254,8 @@ class AdaptiveQueryExecSuite - val localReads = collect(adaptivePlan) { - case read: AQEShuffleReadExec if read.isLocalRead => read - } -- assert(localReads.length == 2) +@@ -200,7 +213,8 @@ class AdaptiveQueryExecSuite + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) +- checkNumLocalShuffleReads(adaptivePlan) + // Comet shuffle changes shuffle metrics -+ assert(localReads.length == 1) - val localShuffleRDD0 = localReads(0).execute().asInstanceOf[ShuffledRowRDD] - val localShuffleRDD1 = localReads(1).execute().asInstanceOf[ShuffledRowRDD] - // The pre-shuffle partition size is [0, 0, 0, 72, 0] -@@ -273,7 +287,8 @@ class AdaptiveQueryExecSuite ++ // checkNumLocalShuffleReads(adaptivePlan) + } + } + +@@ -227,7 +241,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("Reuse the parallelism of coalesced shuffle in local shuffle read") { ++ test("Reuse the parallelism of coalesced shuffle in local shuffle read", ++ IgnoreComet("Comet shuffle changes shuffle partition size")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", +@@ -259,7 +274,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("Reuse the default parallelism in local shuffle read") { ++ test("Reuse the default parallelism in local shuffle read", ++ IgnoreComet("Comet shuffle changes shuffle partition size")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", +@@ -273,7 +289,8 @@ class AdaptiveQueryExecSuite val localReads = collect(adaptivePlan) { case read: AQEShuffleReadExec if read.isLocalRead => read } @@ -1248,7 +1268,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 val localShuffleRDD0 = localReads(0).execute().asInstanceOf[ShuffledRowRDD] val localShuffleRDD1 = localReads(1).execute().asInstanceOf[ShuffledRowRDD] // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 -@@ -298,7 +313,7 @@ class AdaptiveQueryExecSuite +@@ -298,7 +315,7 @@ class AdaptiveQueryExecSuite .groupBy($"a").count() checkAnswer(testDf, Seq()) val plan = testDf.queryExecution.executedPlan @@ -1257,7 +1277,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 val coalescedReads = collect(plan) { case r: AQEShuffleReadExec => r } -@@ -322,7 +337,7 @@ class AdaptiveQueryExecSuite +@@ -322,7 +339,7 @@ class AdaptiveQueryExecSuite } } @@ -1266,7 +1286,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -337,7 +352,7 @@ class AdaptiveQueryExecSuite +@@ -337,7 +354,7 @@ class AdaptiveQueryExecSuite } } @@ -1275,7 +1295,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -353,7 +368,7 @@ class AdaptiveQueryExecSuite +@@ -353,7 +370,7 @@ class AdaptiveQueryExecSuite } } @@ -1284,7 +1304,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -398,7 +413,7 @@ class AdaptiveQueryExecSuite +@@ -398,7 +415,7 @@ class AdaptiveQueryExecSuite } } @@ -1293,7 +1313,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -443,7 +458,7 @@ class AdaptiveQueryExecSuite +@@ -443,7 +460,7 @@ class AdaptiveQueryExecSuite } } @@ -1302,7 +1322,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { -@@ -508,7 +523,7 @@ class AdaptiveQueryExecSuite +@@ -508,7 +525,7 @@ class AdaptiveQueryExecSuite } } @@ -1311,7 +1331,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -539,7 +554,9 @@ class AdaptiveQueryExecSuite +@@ -539,7 +556,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1322,7 +1342,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) -@@ -560,7 +577,9 @@ class AdaptiveQueryExecSuite +@@ -560,7 +579,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1333,7 +1353,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.isEmpty) -@@ -569,7 +588,8 @@ class AdaptiveQueryExecSuite +@@ -569,7 +590,8 @@ class AdaptiveQueryExecSuite } } @@ -1343,7 +1363,17 @@ index 593bd7bb4ba..c736e1fcc54 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", -@@ -786,7 +806,8 @@ class AdaptiveQueryExecSuite +@@ -664,7 +686,8 @@ class AdaptiveQueryExecSuite + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + // There is still a SMJ, and its two shuffles can't apply local read. +- checkNumLocalShuffleReads(adaptivePlan, 2) ++ // Comet shuffle changes shuffle metrics ++ // checkNumLocalShuffleReads(adaptivePlan, 2) + } + } + +@@ -786,7 +809,8 @@ class AdaptiveQueryExecSuite } } @@ -1353,7 +1383,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { joinHint => def getJoinNode(plan: SparkPlan): Seq[ShuffledJoin] = if (joinHint == "SHUFFLE_MERGE") { findTopLevelSortMergeJoin(plan) -@@ -1004,7 +1025,8 @@ class AdaptiveQueryExecSuite +@@ -1004,7 +1028,8 @@ class AdaptiveQueryExecSuite } } @@ -1363,7 +1393,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT key FROM testData GROUP BY key") -@@ -1600,6 +1622,7 @@ class AdaptiveQueryExecSuite +@@ -1600,6 +1625,7 @@ class AdaptiveQueryExecSuite "SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") assert(collect(adaptivePlan) { case s: ShuffleExchangeExec => s @@ -1371,7 +1401,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 }.length == 1) } } -@@ -1679,7 +1702,8 @@ class AdaptiveQueryExecSuite +@@ -1679,7 +1705,8 @@ class AdaptiveQueryExecSuite } } @@ -1381,7 +1411,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 def hasRepartitionShuffle(plan: SparkPlan): Boolean = { find(plan) { case s: ShuffleExchangeLike => -@@ -1864,6 +1888,9 @@ class AdaptiveQueryExecSuite +@@ -1864,6 +1891,9 @@ class AdaptiveQueryExecSuite def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { assert(collect(ds.queryExecution.executedPlan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1391,7 +1421,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 }.size == 1) ds.collect() val plan = ds.queryExecution.executedPlan -@@ -1872,6 +1899,9 @@ class AdaptiveQueryExecSuite +@@ -1872,6 +1902,9 @@ class AdaptiveQueryExecSuite }.isEmpty) assert(collect(plan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1401,7 +1431,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 }.size == 1) checkAnswer(ds, testData) } -@@ -2028,7 +2058,8 @@ class AdaptiveQueryExecSuite +@@ -2028,7 +2061,8 @@ class AdaptiveQueryExecSuite } } @@ -1411,7 +1441,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 withTempView("t1", "t2") { def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { Seq("100", "100000").foreach { size => -@@ -2114,7 +2145,8 @@ class AdaptiveQueryExecSuite +@@ -2114,7 +2148,8 @@ class AdaptiveQueryExecSuite } } @@ -1421,7 +1451,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", -@@ -2214,6 +2246,7 @@ class AdaptiveQueryExecSuite +@@ -2214,6 +2249,7 @@ class AdaptiveQueryExecSuite s"JOIN skewData2 ON key1 = key2 GROUP BY key1") val shuffles1 = collect(adaptive1) { case s: ShuffleExchangeExec => s @@ -1429,7 +1459,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 } assert(shuffles1.size == 3) // shuffles1.head is the top-level shuffle under the Aggregate operator -@@ -2227,6 +2260,7 @@ class AdaptiveQueryExecSuite +@@ -2227,6 +2263,7 @@ class AdaptiveQueryExecSuite s"JOIN skewData2 ON key1 = key2") val shuffles2 = collect(adaptive2) { case s: ShuffleExchangeExec => s @@ -1437,7 +1467,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 } if (hasRequiredDistribution) { assert(shuffles2.size == 3) -@@ -2260,7 +2294,8 @@ class AdaptiveQueryExecSuite +@@ -2260,7 +2297,8 @@ class AdaptiveQueryExecSuite } } @@ -1447,7 +1477,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 CostEvaluator.instantiate( classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, spark.sparkContext.getConf) intercept[IllegalArgumentException] { -@@ -2404,6 +2439,7 @@ class AdaptiveQueryExecSuite +@@ -2404,6 +2442,7 @@ class AdaptiveQueryExecSuite val (_, adaptive) = runAdaptiveAndVerifyResult(query) assert(adaptive.collect { case sort: SortExec => sort @@ -1455,7 +1485,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 }.size == 1) val read = collect(adaptive) { case read: AQEShuffleReadExec => read -@@ -2421,7 +2457,8 @@ class AdaptiveQueryExecSuite +@@ -2421,7 +2460,8 @@ class AdaptiveQueryExecSuite } } @@ -1465,7 +1495,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", -@@ -2534,6 +2571,7 @@ class AdaptiveQueryExecSuite +@@ -2534,6 +2574,7 @@ class AdaptiveQueryExecSuite "JOIN skewData3 ON value2 = value3") val shuffles1 = collect(adaptive1) { case s: ShuffleExchangeExec => s @@ -1473,7 +1503,7 @@ index 593bd7bb4ba..c736e1fcc54 100644 } assert(shuffles1.size == 4) val smj1 = findTopLevelSortMergeJoin(adaptive1) -@@ -2545,6 +2583,7 @@ class AdaptiveQueryExecSuite +@@ -2545,6 +2586,7 @@ class AdaptiveQueryExecSuite "JOIN skewData3 ON value1 = value3") val shuffles2 = collect(adaptive2) { case s: ShuffleExchangeExec => s From 1e560d35611ee38b86a3c1c1fe3dab47483d824f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 3 May 2024 15:14:45 -0700 Subject: [PATCH 23/30] Update diff --- dev/diffs/3.4.2.diff | 36 +++++++++++++++++++++++++++++------- 1 file changed, 29 insertions(+), 7 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index eba47b690f..e33c9ecd60 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -2400,7 +2400,7 @@ index abe606ad9c1..2d930b64cca 100644 val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index dd55fcfe42c..cc18147d17a 100644 +index dd55fcfe42c..293e9dc2986 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest @@ -2424,7 +2424,7 @@ index dd55fcfe42c..cc18147d17a 100644 } } -@@ -242,6 +247,23 @@ private[sql] trait SQLTestUtilsBase +@@ -242,6 +247,32 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -2437,6 +2437,15 @@ index dd55fcfe42c..cc18147d17a 100644 + } + + /** ++ * Whether to enable ansi mode This is only effective when ++ * [[isCometEnabled]] returns true. ++ */ ++ protected def enableCometAnsiMode: Boolean = { ++ val v = System.getenv("ENABLE_COMET_ANSI_MODE") ++ v != null && v.toBoolean ++ } ++ ++ /** + * Whether Spark should only apply Comet scan optimization. This is only effective when + * [[isCometEnabled]] returns true. + */ @@ -2448,7 +2457,7 @@ index dd55fcfe42c..cc18147d17a 100644 protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -434,6 +456,8 @@ private[sql] trait SQLTestUtilsBase +@@ -434,6 +465,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -2458,10 +2467,10 @@ index dd55fcfe42c..cc18147d17a 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index ed2e309fa07..f51a2395a53 100644 +index ed2e309fa07..e071fc44960 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -@@ -74,6 +74,22 @@ trait SharedSparkSessionBase +@@ -74,6 +74,28 @@ trait SharedSparkSessionBase // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) @@ -2480,6 +2489,12 @@ index ed2e309fa07..f51a2395a53 100644 + .set("spark.comet.exec.shuffle.enabled", "true") + .set("spark.comet.memoryOverhead", "10g") + } ++ ++ if (enableCometAnsiMode) { ++ conf ++ .set("spark.sql.ansi.enabled", "true") ++ .set("spark.comet.ansi.enabled", "true") ++ } + } conf.set( StaticSQLConf.WAREHOUSE_PATH, @@ -2534,10 +2549,10 @@ index 1966e1e64fd..cde97a0aafe 100644 spark.sql( """ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala -index 07361cfdce9..c5d94c92e32 100644 +index 07361cfdce9..25b0dc3ef7e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala -@@ -55,25 +55,46 @@ object TestHive +@@ -55,25 +55,53 @@ object TestHive new SparkContext( System.getProperty("spark.sql.test.master", "local[1]"), "TestSQLContext", @@ -2594,6 +2609,13 @@ index 07361cfdce9..c5d94c92e32 100644 + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + .set("spark.comet.exec.shuffle.enabled", "true") + } ++ ++ val a = System.getenv("ENABLE_COMET_ANSI_MODE") ++ if (a != null && a.toBoolean) { ++ conf ++ .set("spark.sql.ansi.enabled", "true") ++ .set("spark.comet.ansi.enabled", "true") ++ } + } + conf From b37070dd275de3c2e4c58ff2d8d6ae82be9ad1d1 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 4 May 2024 14:19:57 -0700 Subject: [PATCH 24/30] Fix memory leak --- dev/diffs/3.4.2.diff | 11 +---------- .../apache/comet/CometSparkSessionExtensions.scala | 4 +++- .../scala/org/apache/spark/sql/CometTestBase.scala | 1 + 3 files changed, 5 insertions(+), 11 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index e33c9ecd60..86a28c2556 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -1149,7 +1149,7 @@ index ac710c32296..88a5329e74e 100644 val df = spark.read.parquet(path).selectExpr(projection: _*) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -index 593bd7bb4ba..f39fe59f36b 100644 +index 593bd7bb4ba..2518d715154 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -26,9 +26,11 @@ import org.scalatest.time.SpanSugar._ @@ -1268,15 +1268,6 @@ index 593bd7bb4ba..f39fe59f36b 100644 val localShuffleRDD0 = localReads(0).execute().asInstanceOf[ShuffledRowRDD] val localShuffleRDD1 = localReads(1).execute().asInstanceOf[ShuffledRowRDD] // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 -@@ -298,7 +315,7 @@ class AdaptiveQueryExecSuite - .groupBy($"a").count() - checkAnswer(testDf, Seq()) - val plan = testDf.queryExecution.executedPlan -- assert(find(plan)(_.isInstanceOf[SortMergeJoinExec]).isDefined) -+ assert(find(plan)(_.isInstanceOf[CometSortMergeJoinExec]).isDefined) - val coalescedReads = collect(plan) { - case r: AQEShuffleReadExec => r - } @@ -322,7 +339,7 @@ class AdaptiveQueryExecSuite } } diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 1e78013c7b..083c1062a4 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -916,7 +916,9 @@ object CometSparkSessionExtensions extends Logging { private[comet] def isCometShuffleEnabled(conf: SQLConf): Boolean = COMET_EXEC_SHUFFLE_ENABLED.get(conf) && (conf.contains("spark.shuffle.manager") && conf.getConfString("spark.shuffle.manager") == - "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") && + // TODO: AQE coalesce partitions feature causes Comet columnar shuffle memory leak + !conf.coalesceShufflePartitionsEnabled private[comet] def isCometScanEnabled(conf: SQLConf): Boolean = { COMET_SCAN_ENABLED.get(conf) diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index 8fda13617e..1ed447dc34 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -76,6 +76,7 @@ abstract class CometTestBase conf.set(MEMORY_OFFHEAP_SIZE.key, "2g") conf.set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "1g") conf.set(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key, "1g") + conf.set(SQLConf.COALESCE_PARTITIONS_ENABLED.key, "false") conf.set(CometConf.COMET_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key, "true") From 557b7535ffe69463184e0b4308606aa3ad38cb08 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 4 May 2024 21:17:08 -0700 Subject: [PATCH 25/30] Update plan stability --- .../approved-plans-v1_4/q1/explain.txt | 192 ++- .../approved-plans-v1_4/q1/simplified.txt | 82 +- .../approved-plans-v1_4/q10/explain.txt | 126 +- .../approved-plans-v1_4/q10/simplified.txt | 128 +- .../approved-plans-v1_4/q11/explain.txt | 326 ++-- .../approved-plans-v1_4/q11/simplified.txt | 172 +-- .../approved-plans-v1_4/q12/explain.txt | 96 +- .../approved-plans-v1_4/q12/simplified.txt | 64 +- .../approved-plans-v1_4/q13/explain.txt | 102 +- .../approved-plans-v1_4/q13/simplified.txt | 100 +- .../approved-plans-v1_4/q14a/explain.txt | 536 +++---- .../approved-plans-v1_4/q14a/simplified.txt | 414 +++--- .../approved-plans-v1_4/q14b/explain.txt | 480 +++--- .../approved-plans-v1_4/q14b/simplified.txt | 356 +++-- .../approved-plans-v1_4/q15/explain.txt | 80 +- .../approved-plans-v1_4/q15/simplified.txt | 70 +- .../approved-plans-v1_4/q16/explain.txt | 164 +- .../approved-plans-v1_4/q16/simplified.txt | 116 +- .../approved-plans-v1_4/q17/explain.txt | 138 +- .../approved-plans-v1_4/q17/simplified.txt | 126 +- .../approved-plans-v1_4/q18/explain.txt | 122 +- .../approved-plans-v1_4/q18/simplified.txt | 116 +- .../approved-plans-v1_4/q19/explain.txt | 94 +- .../approved-plans-v1_4/q19/simplified.txt | 88 +- .../approved-plans-v1_4/q2/explain.txt | 140 +- .../approved-plans-v1_4/q2/simplified.txt | 101 +- .../approved-plans-v1_4/q20/explain.txt | 96 +- .../approved-plans-v1_4/q20/simplified.txt | 64 +- .../approved-plans-v1_4/q21/explain.txt | 82 +- .../approved-plans-v1_4/q21/simplified.txt | 70 +- .../approved-plans-v1_4/q22/explain.txt | 82 +- .../approved-plans-v1_4/q22/simplified.txt | 72 +- .../approved-plans-v1_4/q23a/explain.txt | 428 +++--- .../approved-plans-v1_4/q23a/simplified.txt | 301 ++-- .../approved-plans-v1_4/q23b/explain.txt | 528 +++---- .../approved-plans-v1_4/q23b/simplified.txt | 364 +++-- .../approved-plans-v1_4/q24a/explain.txt | 358 ++--- .../approved-plans-v1_4/q24a/simplified.txt | 188 +-- .../approved-plans-v1_4/q24b/explain.txt | 358 ++--- .../approved-plans-v1_4/q24b/simplified.txt | 188 +-- .../approved-plans-v1_4/q25/explain.txt | 138 +- .../approved-plans-v1_4/q25/simplified.txt | 126 +- .../approved-plans-v1_4/q26/explain.txt | 96 +- .../approved-plans-v1_4/q26/simplified.txt | 90 +- .../approved-plans-v1_4/q27/explain.txt | 96 +- .../approved-plans-v1_4/q27/simplified.txt | 90 +- .../approved-plans-v1_4/q28/explain.txt | 346 ++--- .../approved-plans-v1_4/q28/simplified.txt | 132 +- .../approved-plans-v1_4/q29/explain.txt | 156 +- .../approved-plans-v1_4/q29/simplified.txt | 134 +- .../approved-plans-v1_4/q3/explain.txt | 58 +- .../approved-plans-v1_4/q3/simplified.txt | 52 +- .../approved-plans-v1_4/q30/explain.txt | 216 ++- .../approved-plans-v1_4/q30/simplified.txt | 106 +- .../approved-plans-v1_4/q31/explain.txt | 450 +++--- .../approved-plans-v1_4/q31/simplified.txt | 300 ++-- .../approved-plans-v1_4/q32/explain.txt | 122 +- .../approved-plans-v1_4/q32/simplified.txt | 98 +- .../approved-plans-v1_4/q33/explain.txt | 260 ++-- .../approved-plans-v1_4/q33/simplified.txt | 200 ++- .../approved-plans-v1_4/q34/explain.txt | 126 +- .../approved-plans-v1_4/q34/simplified.txt | 108 +- .../approved-plans-v1_4/q35/explain.txt | 124 +- .../approved-plans-v1_4/q35/simplified.txt | 126 +- .../approved-plans-v1_4/q36/explain.txt | 112 +- .../approved-plans-v1_4/q36/simplified.txt | 88 +- .../approved-plans-v1_4/q37/explain.txt | 90 +- .../approved-plans-v1_4/q37/simplified.txt | 88 +- .../approved-plans-v1_4/q38/explain.txt | 228 ++- .../approved-plans-v1_4/q38/simplified.txt | 160 +- .../approved-plans-v1_4/q39a/explain.txt | 208 ++- .../approved-plans-v1_4/q39a/simplified.txt | 152 +- .../approved-plans-v1_4/q39b/explain.txt | 208 ++- .../approved-plans-v1_4/q39b/simplified.txt | 152 +- .../approved-plans-v1_4/q4/explain.txt | 486 +++--- .../approved-plans-v1_4/q4/simplified.txt | 260 ++-- .../approved-plans-v1_4/q40/explain.txt | 167 +-- .../approved-plans-v1_4/q40/simplified.txt | 98 +- .../approved-plans-v1_4/q41/explain.txt | 94 +- .../approved-plans-v1_4/q41/simplified.txt | 48 +- .../approved-plans-v1_4/q42/explain.txt | 58 +- .../approved-plans-v1_4/q42/simplified.txt | 52 +- .../approved-plans-v1_4/q43/explain.txt | 58 +- .../approved-plans-v1_4/q43/simplified.txt | 52 +- .../approved-plans-v1_4/q44/explain.txt | 260 ++-- .../approved-plans-v1_4/q44/simplified.txt | 59 +- .../approved-plans-v1_4/q45/explain.txt | 108 +- .../approved-plans-v1_4/q45/simplified.txt | 102 +- .../approved-plans-v1_4/q46/explain.txt | 130 +- .../approved-plans-v1_4/q46/simplified.txt | 86 +- .../approved-plans-v1_4/q47/explain.txt | 202 ++- .../approved-plans-v1_4/q47/simplified.txt | 100 +- .../approved-plans-v1_4/q48/explain.txt | 90 +- .../approved-plans-v1_4/q48/simplified.txt | 84 +- .../approved-plans-v1_4/q49/explain.txt | 364 ++--- .../approved-plans-v1_4/q49/simplified.txt | 256 ++-- .../approved-plans-v1_4/q5/explain.txt | 298 ++-- .../approved-plans-v1_4/q5/simplified.txt | 218 ++- .../approved-plans-v1_4/q50/explain.txt | 92 +- .../approved-plans-v1_4/q50/simplified.txt | 78 +- .../approved-plans-v1_4/q51/explain.txt | 231 ++- .../approved-plans-v1_4/q51/simplified.txt | 138 +- .../approved-plans-v1_4/q52/explain.txt | 58 +- .../approved-plans-v1_4/q52/simplified.txt | 52 +- .../approved-plans-v1_4/q53/explain.txt | 114 +- .../approved-plans-v1_4/q53/simplified.txt | 84 +- .../approved-plans-v1_4/q54/explain.txt | 308 ++-- .../approved-plans-v1_4/q54/simplified.txt | 206 ++- .../approved-plans-v1_4/q55/explain.txt | 58 +- .../approved-plans-v1_4/q55/simplified.txt | 52 +- .../approved-plans-v1_4/q56/explain.txt | 260 ++-- .../approved-plans-v1_4/q56/simplified.txt | 200 ++- .../approved-plans-v1_4/q57/explain.txt | 202 ++- .../approved-plans-v1_4/q57/simplified.txt | 100 +- .../approved-plans-v1_4/q58/explain.txt | 258 ++-- .../approved-plans-v1_4/q58/simplified.txt | 158 +- .../approved-plans-v1_4/q59/explain.txt | 162 +- .../approved-plans-v1_4/q59/simplified.txt | 41 +- .../approved-plans-v1_4/q6/explain.txt | 252 ++-- .../approved-plans-v1_4/q6/simplified.txt | 126 +- .../approved-plans-v1_4/q60/explain.txt | 260 ++-- .../approved-plans-v1_4/q60/simplified.txt | 200 ++- .../approved-plans-v1_4/q61/explain.txt | 226 ++- .../approved-plans-v1_4/q61/simplified.txt | 170 ++- .../approved-plans-v1_4/q62/explain.txt | 80 +- .../approved-plans-v1_4/q62/simplified.txt | 74 +- .../approved-plans-v1_4/q63/explain.txt | 114 +- .../approved-plans-v1_4/q63/simplified.txt | 84 +- .../approved-plans-v1_4/q64/explain.txt | 1317 ++++++++--------- .../approved-plans-v1_4/q64/simplified.txt | 395 ++--- .../approved-plans-v1_4/q65/explain.txt | 178 +-- .../approved-plans-v1_4/q65/simplified.txt | 82 +- .../approved-plans-v1_4/q66/explain.txt | 200 ++- .../approved-plans-v1_4/q66/simplified.txt | 154 +- .../approved-plans-v1_4/q67/explain.txt | 110 +- .../approved-plans-v1_4/q67/simplified.txt | 86 +- .../approved-plans-v1_4/q68/explain.txt | 130 +- .../approved-plans-v1_4/q68/simplified.txt | 86 +- .../approved-plans-v1_4/q69/explain.txt | 124 +- .../approved-plans-v1_4/q69/simplified.txt | 124 +- .../approved-plans-v1_4/q7/explain.txt | 96 +- .../approved-plans-v1_4/q7/simplified.txt | 90 +- .../approved-plans-v1_4/q70/explain.txt | 178 +-- .../approved-plans-v1_4/q70/simplified.txt | 134 +- .../approved-plans-v1_4/q71/explain.txt | 130 +- .../approved-plans-v1_4/q71/simplified.txt | 132 +- .../approved-plans-v1_4/q72/explain.txt | 233 +-- .../approved-plans-v1_4/q72/simplified.txt | 184 +-- .../approved-plans-v1_4/q73/explain.txt | 126 +- .../approved-plans-v1_4/q73/simplified.txt | 108 +- .../approved-plans-v1_4/q74/explain.txt | 322 ++-- .../approved-plans-v1_4/q74/simplified.txt | 172 +-- .../approved-plans-v1_4/q75/explain.txt | 850 ++++++----- .../approved-plans-v1_4/q75/simplified.txt | 383 +++-- .../approved-plans-v1_4/q76/explain.txt | 92 +- .../approved-plans-v1_4/q76/simplified.txt | 100 +- .../approved-plans-v1_4/q77/explain.txt | 402 +++-- .../approved-plans-v1_4/q77/simplified.txt | 278 ++-- .../approved-plans-v1_4/q78/explain.txt | 373 +++-- .../approved-plans-v1_4/q78/simplified.txt | 190 +-- .../approved-plans-v1_4/q79/explain.txt | 106 +- .../approved-plans-v1_4/q79/simplified.txt | 74 +- .../approved-plans-v1_4/q8/explain.txt | 170 +-- .../approved-plans-v1_4/q8/simplified.txt | 136 +- .../approved-plans-v1_4/q80/explain.txt | 521 ++++--- .../approved-plans-v1_4/q80/simplified.txt | 266 ++-- .../approved-plans-v1_4/q81/explain.txt | 212 ++- .../approved-plans-v1_4/q81/simplified.txt | 106 +- .../approved-plans-v1_4/q82/explain.txt | 90 +- .../approved-plans-v1_4/q82/simplified.txt | 88 +- .../approved-plans-v1_4/q83/explain.txt | 248 ++-- .../approved-plans-v1_4/q83/simplified.txt | 158 +- .../approved-plans-v1_4/q85/explain.txt | 130 +- .../approved-plans-v1_4/q85/simplified.txt | 120 +- .../approved-plans-v1_4/q86/explain.txt | 98 +- .../approved-plans-v1_4/q86/simplified.txt | 70 +- .../approved-plans-v1_4/q87/explain.txt | 228 ++- .../approved-plans-v1_4/q87/simplified.txt | 160 +- .../approved-plans-v1_4/q88/explain.txt | 728 +++++---- .../approved-plans-v1_4/q88/simplified.txt | 286 ++-- .../approved-plans-v1_4/q89/explain.txt | 112 +- .../approved-plans-v1_4/q89/simplified.txt | 82 +- .../approved-plans-v1_4/q9/explain.txt | 210 +-- .../approved-plans-v1_4/q9/simplified.txt | 90 +- .../approved-plans-v1_4/q90/explain.txt | 168 +-- .../approved-plans-v1_4/q90/simplified.txt | 120 +- .../approved-plans-v1_4/q91/explain.txt | 136 +- .../approved-plans-v1_4/q91/simplified.txt | 126 +- .../approved-plans-v1_4/q92/explain.txt | 122 +- .../approved-plans-v1_4/q92/simplified.txt | 98 +- .../approved-plans-v1_4/q93/explain.txt | 119 +- .../approved-plans-v1_4/q93/simplified.txt | 66 +- .../approved-plans-v1_4/q94/explain.txt | 164 +- .../approved-plans-v1_4/q94/simplified.txt | 116 +- .../approved-plans-v1_4/q95/explain.txt | 252 ++-- .../approved-plans-v1_4/q95/simplified.txt | 171 +-- .../approved-plans-v1_4/q96/explain.txt | 70 +- .../approved-plans-v1_4/q96/simplified.txt | 68 +- .../approved-plans-v1_4/q97/explain.txt | 151 +- .../approved-plans-v1_4/q97/simplified.txt | 88 +- .../approved-plans-v1_4/q98/explain.txt | 118 +- .../approved-plans-v1_4/q98/simplified.txt | 90 +- .../approved-plans-v1_4/q99/explain.txt | 80 +- .../approved-plans-v1_4/q99/simplified.txt | 74 +- .../approved-plans-v2_7/q10a/explain.txt | 122 +- .../approved-plans-v2_7/q10a/simplified.txt | 130 +- .../approved-plans-v2_7/q11/explain.txt | 322 ++-- .../approved-plans-v2_7/q11/simplified.txt | 172 +-- .../approved-plans-v2_7/q12/explain.txt | 96 +- .../approved-plans-v2_7/q12/simplified.txt | 64 +- .../approved-plans-v2_7/q14/explain.txt | 480 +++--- .../approved-plans-v2_7/q14/simplified.txt | 356 +++-- .../approved-plans-v2_7/q14a/explain.txt | 704 ++++----- .../approved-plans-v2_7/q14a/simplified.txt | 526 ++++--- .../approved-plans-v2_7/q18a/explain.txt | 596 ++++---- .../approved-plans-v2_7/q18a/simplified.txt | 390 +++-- .../approved-plans-v2_7/q20/explain.txt | 96 +- .../approved-plans-v2_7/q20/simplified.txt | 64 +- .../approved-plans-v2_7/q22/explain.txt | 80 +- .../approved-plans-v2_7/q22/simplified.txt | 70 +- .../approved-plans-v2_7/q22a/explain.txt | 226 ++- .../approved-plans-v2_7/q22a/simplified.txt | 138 +- .../approved-plans-v2_7/q24/explain.txt | 378 +++-- .../approved-plans-v2_7/q24/simplified.txt | 224 +-- .../approved-plans-v2_7/q27a/explain.txt | 286 ++-- .../approved-plans-v2_7/q27a/simplified.txt | 202 ++- .../approved-plans-v2_7/q34/explain.txt | 126 +- .../approved-plans-v2_7/q34/simplified.txt | 108 +- .../approved-plans-v2_7/q35/explain.txt | 124 +- .../approved-plans-v2_7/q35/simplified.txt | 126 +- .../approved-plans-v2_7/q35a/explain.txt | 120 +- .../approved-plans-v2_7/q35a/simplified.txt | 128 +- .../approved-plans-v2_7/q36a/explain.txt | 198 +-- .../approved-plans-v2_7/q36a/simplified.txt | 122 +- .../approved-plans-v2_7/q47/explain.txt | 202 ++- .../approved-plans-v2_7/q47/simplified.txt | 100 +- .../approved-plans-v2_7/q49/explain.txt | 364 ++--- .../approved-plans-v2_7/q49/simplified.txt | 256 ++-- .../approved-plans-v2_7/q51a/explain.txt | 389 ++--- .../approved-plans-v2_7/q51a/simplified.txt | 169 +-- .../approved-plans-v2_7/q57/explain.txt | 202 ++- .../approved-plans-v2_7/q57/simplified.txt | 100 +- .../approved-plans-v2_7/q5a/explain.txt | 652 ++++---- .../approved-plans-v2_7/q5a/simplified.txt | 450 +++--- .../approved-plans-v2_7/q6/explain.txt | 252 ++-- .../approved-plans-v2_7/q6/simplified.txt | 126 +- .../approved-plans-v2_7/q64/explain.txt | 1317 ++++++++--------- .../approved-plans-v2_7/q64/simplified.txt | 395 ++--- .../approved-plans-v2_7/q67a/explain.txt | 384 ++--- .../approved-plans-v2_7/q67a/simplified.txt | 252 ++-- .../approved-plans-v2_7/q70a/explain.txt | 262 ++-- .../approved-plans-v2_7/q70a/simplified.txt | 170 +-- .../approved-plans-v2_7/q72/explain.txt | 233 +-- .../approved-plans-v2_7/q72/simplified.txt | 184 +-- .../approved-plans-v2_7/q74/explain.txt | 322 ++-- .../approved-plans-v2_7/q74/simplified.txt | 172 +-- .../approved-plans-v2_7/q75/explain.txt | 850 ++++++----- .../approved-plans-v2_7/q75/simplified.txt | 383 +++-- .../approved-plans-v2_7/q77a/explain.txt | 496 +++---- .../approved-plans-v2_7/q77a/simplified.txt | 342 ++--- .../approved-plans-v2_7/q78/explain.txt | 373 +++-- .../approved-plans-v2_7/q78/simplified.txt | 190 +-- .../approved-plans-v2_7/q80a/explain.txt | 615 ++++---- .../approved-plans-v2_7/q80a/simplified.txt | 386 ++--- .../approved-plans-v2_7/q86a/explain.txt | 184 +-- .../approved-plans-v2_7/q86a/simplified.txt | 108 +- .../approved-plans-v2_7/q98/explain.txt | 112 +- .../approved-plans-v2_7/q98/simplified.txt | 86 +- 268 files changed, 25029 insertions(+), 28380 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt index e1651dccbd..b0ea6bed8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt @@ -1,50 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (46) -+- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Filter (12) - : : : +- * HashAggregate (11) - : : : +- * ColumnarToRow (10) - : : : +- CometColumnarExchange (9) - : : : +- RowToColumnar (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_returns (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (30) - : : +- * Filter (29) - : : +- * HashAggregate (28) - : : +- * ColumnarToRow (27) - : : +- CometColumnarExchange (26) - : : +- RowToColumnar (25) - : : +- * HashAggregate (24) - : : +- * HashAggregate (23) - : : +- * ColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- RowToColumnar (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * ColumnarToRow (15) - : : : +- CometFilter (14) - : : : +- CometScan parquet spark_catalog.default.store_returns (13) - : : +- ReusedExchange (16) - : +- BroadcastExchange (37) - : +- * ColumnarToRow (36) - : +- CometProject (35) - : +- CometFilter (34) - : +- CometScan parquet spark_catalog.default.store (33) - +- BroadcastExchange (43) - +- * ColumnarToRow (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.customer (40) +TakeOrderedAndProject (40) ++- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Filter (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_returns (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (24) + : : +- * Filter (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * HashAggregate (19) + : : +- Exchange (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * ColumnarToRow (13) + : : : +- CometFilter (12) + : : : +- CometScan parquet spark_catalog.default.store_returns (11) + : : +- ReusedExchange (14) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.store (27) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer (34) (unknown) Scan parquet spark_catalog.default.store_returns @@ -62,7 +56,7 @@ Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) (3) ColumnarToRow [codegen id : 2] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -(4) ReusedExchange [Reuses operator id: 51] +(4) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 2] @@ -82,24 +76,18 @@ Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum#7] Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -(8) RowToColumnar +(8) Exchange Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) CometColumnarExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(10) ColumnarToRow [codegen id : 9] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] - -(11) HashAggregate [codegen id : 9] +(9) HashAggregate [codegen id : 9] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(12) Filter [codegen id : 9] +(10) Filter [codegen id : 9] Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] Condition : isnotnull(ctr_total_return#12) @@ -111,89 +99,77 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Condition : isnotnull(sr_store_sk#2) -(15) ColumnarToRow [codegen id : 4] +(13) ColumnarToRow [codegen id : 4] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -(16) ReusedExchange [Reuses operator id: 51] +(14) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#6] -(17) BroadcastHashJoin [codegen id : 4] +(15) BroadcastHashJoin [codegen id : 4] Left keys [1]: [sr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(16) Project [codegen id : 4] Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -(19) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 4] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum#14] Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] -(20) RowToColumnar +(18) Exchange Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) CometColumnarExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(22) ColumnarToRow [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] - -(23) HashAggregate [codegen id : 5] +(19) HashAggregate [codegen id : 5] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] Results [2]: [sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(24) HashAggregate [codegen id : 5] +(20) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#11, ctr_total_return#12] Keys [1]: [ctr_store_sk#11] Functions [1]: [partial_avg(ctr_total_return#12)] Aggregate Attributes [2]: [sum#16, count#17] Results [3]: [ctr_store_sk#11, sum#18, count#19] -(25) RowToColumnar -Input [3]: [ctr_store_sk#11, sum#18, count#19] - -(26) CometColumnarExchange -Input [3]: [ctr_store_sk#11, sum#18, count#19] -Arguments: hashpartitioning(ctr_store_sk#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(27) ColumnarToRow [codegen id : 6] +(21) Exchange Input [3]: [ctr_store_sk#11, sum#18, count#19] +Arguments: hashpartitioning(ctr_store_sk#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(28) HashAggregate [codegen id : 6] +(22) HashAggregate [codegen id : 6] Input [3]: [ctr_store_sk#11, sum#18, count#19] Keys [1]: [ctr_store_sk#11] Functions [1]: [avg(ctr_total_return#12)] Aggregate Attributes [1]: [avg(ctr_total_return#12)#20] Results [2]: [(avg(ctr_total_return#12)#20 * 1.2) AS (avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11 AS ctr_store_sk#11#22] -(29) Filter [codegen id : 6] +(23) Filter [codegen id : 6] Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] Condition : isnotnull((avg(ctr_total_return) * 1.2)#21) -(30) BroadcastExchange +(24) BroadcastExchange Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 9] +(25) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#11] Right keys [1]: [ctr_store_sk#11#22] Join type: Inner Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#21) -(32) Project [codegen id : 9] +(26) Project [codegen id : 9] Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] @@ -204,28 +180,28 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(34) CometFilter +(28) CometFilter Input [2]: [s_store_sk#23, s_state#24] Condition : ((isnotnull(s_state#24) AND (s_state#24 = TN)) AND isnotnull(s_store_sk#23)) -(35) CometProject +(29) CometProject Input [2]: [s_store_sk#23, s_state#24] Arguments: [s_store_sk#23], [s_store_sk#23] -(36) ColumnarToRow [codegen id : 7] +(30) ColumnarToRow [codegen id : 7] Input [1]: [s_store_sk#23] -(37) BroadcastExchange +(31) BroadcastExchange Input [1]: [s_store_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#11] Right keys [1]: [s_store_sk#23] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(33) Project [codegen id : 9] Output [1]: [ctr_customer_sk#10] Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#23] @@ -236,39 +212,39 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(41) CometFilter +(35) CometFilter Input [2]: [c_customer_sk#25, c_customer_id#26] Condition : isnotnull(c_customer_sk#25) -(42) ColumnarToRow [codegen id : 8] +(36) ColumnarToRow [codegen id : 8] Input [2]: [c_customer_sk#25, c_customer_id#26] -(43) BroadcastExchange +(37) BroadcastExchange Input [2]: [c_customer_sk#25, c_customer_id#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(44) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_customer_sk#10] Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(39) Project [codegen id : 9] Output [1]: [c_customer_id#26] Input [3]: [ctr_customer_sk#10, c_customer_sk#25, c_customer_id#26] -(46) TakeOrderedAndProject +(40) TakeOrderedAndProject Input [1]: [c_customer_id#26] Arguments: 100, [c_customer_id#26 ASC NULLS FIRST], [c_customer_id#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (51) -+- * ColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan parquet spark_catalog.default.date_dim (47) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) (unknown) Scan parquet spark_catalog.default.date_dim @@ -278,21 +254,21 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter +(42) CometFilter Input [2]: [d_date_sk#6, d_year#27] Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2000)) AND isnotnull(d_date_sk#6)) -(49) CometProject +(43) CometProject Input [2]: [d_date_sk#6, d_year#27] Arguments: [d_date_sk#6], [d_date_sk#6] -(50) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(51) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 13 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index 5bf8a8e46d..6d4c0fca7a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -8,55 +8,49 @@ TakeOrderedAndProject [c_customer_id] BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [sr_store_sk,sr_customer_sk] - CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 + InputAdapter + Exchange [sr_customer_sk,sr_store_sk] #1 + WholeStageCodegen (2) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_store_sk,sr_customer_sk] + CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 WholeStageCodegen (6) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_store_sk,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ctr_store_sk] #4 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [sr_customer_sk,sr_store_sk] #5 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [sr_store_sk] - CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 + InputAdapter + Exchange [ctr_store_sk] #4 + WholeStageCodegen (5) + HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] + InputAdapter + Exchange [sr_customer_sk,sr_store_sk] #5 + WholeStageCodegen (4) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_store_sk] + CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (7) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index dda5b40fac..1ea234408b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -1,49 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- * ColumnarToRow (43) - +- CometColumnarExchange (42) - +- RowToColumnar (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer_demographics (34) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * Filter (25) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer_demographics (34) (unknown) Scan parquet spark_catalog.default.customer @@ -70,7 +68,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) ReusedExchange [Reuses operator id: 50] +(6) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#9] (7) BroadcastHashJoin [codegen id : 2] @@ -103,7 +101,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(13) ReusedExchange [Reuses operator id: 50] +(13) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#13] (14) BroadcastHashJoin [codegen id : 4] @@ -136,7 +134,7 @@ ReadSchema: struct (19) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -(20) ReusedExchange [Reuses operator id: 50] +(20) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 6] @@ -234,35 +232,29 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -(41) RowToColumnar +(41) Exchange Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] +Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(42) CometColumnarExchange -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) ColumnarToRow [codegen id : 10] -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] - -(44) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 10] Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#31] Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] -(45) TakeOrderedAndProject +(43) TakeOrderedAndProject Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) (unknown) Scan parquet spark_catalog.default.date_dim @@ -272,18 +264,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(45) CometFilter Input [3]: [d_date_sk#9, d_year#38, d_moy#39] Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 1)) AND (d_moy#39 <= 4)) AND isnotnull(d_date_sk#9)) -(48) CometProject +(46) CometProject Input [3]: [d_date_sk#9, d_year#38, d_moy#39] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(50) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 067c8fb293..89893c831e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -1,77 +1,75 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] WholeStageCodegen (10) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [ca_address_sk] + CometFilter [ca_county,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index bccf5e463d..64f486f71e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt @@ -1,84 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (80) -+- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * ColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- RowToColumnar (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * ColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- RowToColumnar (32) - : : +- * HashAggregate (31) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Project (27) - : : : +- * BroadcastHashJoin Inner BuildRight (26) - : : : :- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.store_sales (22) - : : +- ReusedExchange (28) - : +- BroadcastExchange (57) - : +- * Filter (56) - : +- * HashAggregate (55) - : +- * ColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- RowToColumnar (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * ColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometScan parquet spark_catalog.default.customer (39) - : : +- BroadcastExchange (45) - : : +- * ColumnarToRow (44) - : : +- CometFilter (43) - : : +- CometScan parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (77) - +- * HashAggregate (76) - +- * ColumnarToRow (75) - +- CometColumnarExchange (74) - +- RowToColumnar (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * ColumnarToRow (62) - : : +- CometFilter (61) - : : +- CometScan parquet spark_catalog.default.customer (60) - : +- BroadcastExchange (66) - : +- * ColumnarToRow (65) - : +- CometFilter (64) - : +- CometScan parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) +TakeOrderedAndProject (72) ++- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * Project (34) + : : +- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (25) + : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- BroadcastExchange (23) + : : : +- * ColumnarToRow (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- ReusedExchange (26) + : +- BroadcastExchange (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * HashAggregate (47) + : +- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (43) + : : +- * BroadcastHashJoin Inner BuildRight (42) + : : :- * ColumnarToRow (37) + : : : +- CometFilter (36) + : : : +- CometScan parquet spark_catalog.default.customer (35) + : : +- BroadcastExchange (41) + : : +- * ColumnarToRow (40) + : : +- CometFilter (39) + : : +- CometScan parquet spark_catalog.default.web_sales (38) + : +- ReusedExchange (44) + +- BroadcastExchange (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * ColumnarToRow (56) + : : +- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.customer (54) + : +- BroadcastExchange (60) + : +- * ColumnarToRow (59) + : +- CometFilter (58) + : +- CometScan parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (63) (unknown) Scan parquet spark_catalog.default.customer @@ -124,7 +116,7 @@ Join condition: None Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(10) ReusedExchange [Reuses operator id: 84] +(10) ReusedExchange [Reuses operator id: 76] Output [2]: [d_date_sk#14, d_year#15] (11) BroadcastHashJoin [codegen id : 3] @@ -144,24 +136,18 @@ Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discoun Aggregate Attributes [1]: [sum#16] Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -(14) RowToColumnar +(14) Exchange Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 16] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] - -(17) HashAggregate [codegen id : 16] +(15) HashAggregate [codegen id : 16] Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] -(18) Filter [codegen id : 16] +(16) Filter [codegen id : 16] Input [2]: [customer_id#19, year_total#20] Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) @@ -172,11 +158,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(20) CometFilter +(18) CometFilter Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) -(21) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 6] Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] (unknown) Scan parquet spark_catalog.default.store_sales @@ -187,75 +173,69 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter +(21) CometFilter Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Condition : isnotnull(ss_customer_sk#29) -(24) ColumnarToRow [codegen id : 4] +(22) ColumnarToRow [codegen id : 4] Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -(25) BroadcastExchange +(23) BroadcastExchange Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [c_customer_sk#21] Right keys [1]: [ss_customer_sk#29] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(25) Project [codegen id : 6] Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -(28) ReusedExchange [Reuses operator id: 88] +(26) ReusedExchange [Reuses operator id: 80] Output [2]: [d_date_sk#34, d_year#35] -(29) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#32] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(30) Project [codegen id : 6] +(28) Project [codegen id : 6] Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] -(31) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] Aggregate Attributes [1]: [sum#36] Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -(32) RowToColumnar -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] - -(33) CometColumnarExchange -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) ColumnarToRow [codegen id : 7] +(30) Exchange Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] +Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 7] Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] Results [3]: [c_customer_id#22 AS customer_id#38, c_preferred_cust_flag#25 AS customer_preferred_cust_flag#39, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#40] -(36) BroadcastExchange +(32) BroadcastExchange Input [3]: [customer_id#38, customer_preferred_cust_flag#39, year_total#40] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(33) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#38] Join type: Inner Join condition: None -(38) Project [codegen id : 16] +(34) Project [codegen id : 16] Output [4]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40] Input [5]: [customer_id#19, year_total#20, customer_id#38, customer_preferred_cust_flag#39, year_total#40] @@ -266,11 +246,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(40) CometFilter +(36) CometFilter Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_customer_id#42)) -(41) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 10] Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] (unknown) Scan parquet spark_catalog.default.web_sales @@ -281,79 +261,73 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) CometFilter +(39) CometFilter Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] Condition : isnotnull(ws_bill_customer_sk#49) -(44) ColumnarToRow [codegen id : 8] +(40) ColumnarToRow [codegen id : 8] Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -(45) BroadcastExchange +(41) BroadcastExchange Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(42) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#41] Right keys [1]: [ws_bill_customer_sk#49] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(43) Project [codegen id : 10] Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] Input [12]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -(48) ReusedExchange [Reuses operator id: 84] +(44) ReusedExchange [Reuses operator id: 76] Output [2]: [d_date_sk#54, d_year#55] -(49) BroadcastHashJoin [codegen id : 10] +(45) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#52] Right keys [1]: [d_date_sk#54] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(46) Project [codegen id : 10] Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52, d_date_sk#54, d_year#55] -(51) HashAggregate [codegen id : 10] +(47) HashAggregate [codegen id : 10] Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] Aggregate Attributes [1]: [sum#56] Results [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] -(52) RowToColumnar +(48) Exchange Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] +Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(53) CometColumnarExchange -Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] -Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(54) ColumnarToRow [codegen id : 11] -Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] - -(55) HashAggregate [codegen id : 11] +(49) HashAggregate [codegen id : 11] Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58] Results [2]: [c_customer_id#42 AS customer_id#59, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58,18,2) AS year_total#60] -(56) Filter [codegen id : 11] +(50) Filter [codegen id : 11] Input [2]: [customer_id#59, year_total#60] Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.00)) -(57) BroadcastExchange +(51) BroadcastExchange Input [2]: [customer_id#59, year_total#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(58) BroadcastHashJoin [codegen id : 16] +(52) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#59] Join type: Inner Join condition: None -(59) Project [codegen id : 16] +(53) Project [codegen id : 16] Output [5]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60] Input [6]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, customer_id#59, year_total#60] @@ -364,11 +338,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(61) CometFilter +(55) CometFilter Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] Condition : (isnotnull(c_customer_sk#61) AND isnotnull(c_customer_id#62)) -(62) ColumnarToRow [codegen id : 14] +(56) ColumnarToRow [codegen id : 14] Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] (unknown) Scan parquet spark_catalog.default.web_sales @@ -379,89 +353,83 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) CometFilter +(58) CometFilter Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Condition : isnotnull(ws_bill_customer_sk#69) -(65) ColumnarToRow [codegen id : 12] +(59) ColumnarToRow [codegen id : 12] Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -(66) BroadcastExchange +(60) BroadcastExchange Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(61) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_customer_sk#61] Right keys [1]: [ws_bill_customer_sk#69] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(62) Project [codegen id : 14] Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Input [12]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -(69) ReusedExchange [Reuses operator id: 88] +(63) ReusedExchange [Reuses operator id: 80] Output [2]: [d_date_sk#74, d_year#75] -(70) BroadcastHashJoin [codegen id : 14] +(64) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_sold_date_sk#72] Right keys [1]: [d_date_sk#74] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(65) Project [codegen id : 14] Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] Input [12]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72, d_date_sk#74, d_year#75] -(72) HashAggregate [codegen id : 14] +(66) HashAggregate [codegen id : 14] Input [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] Aggregate Attributes [1]: [sum#76] Results [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] -(73) RowToColumnar -Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] - -(74) CometColumnarExchange -Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] -Arguments: hashpartitioning(c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(75) ColumnarToRow [codegen id : 15] +(67) Exchange Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] +Arguments: hashpartitioning(c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(76) HashAggregate [codegen id : 15] +(68) HashAggregate [codegen id : 15] Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#58] Results [2]: [c_customer_id#62 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#58,18,2) AS year_total#79] -(77) BroadcastExchange +(69) BroadcastExchange Input [2]: [customer_id#78, year_total#79] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(78) BroadcastHashJoin [codegen id : 16] +(70) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#78] Join type: Inner Join condition: (CASE WHEN (year_total#60 > 0.00) THEN (year_total#79 / year_total#60) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END) -(79) Project [codegen id : 16] +(71) Project [codegen id : 16] Output [1]: [customer_preferred_cust_flag#39] Input [7]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60, customer_id#78, year_total#79] -(80) TakeOrderedAndProject +(72) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#39] Arguments: 100, [customer_preferred_cust_flag#39 ASC NULLS FIRST], [customer_preferred_cust_flag#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (84) -+- * ColumnarToRow (83) - +- CometFilter (82) - +- CometScan parquet spark_catalog.default.date_dim (81) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) (unknown) Scan parquet spark_catalog.default.date_dim @@ -471,22 +439,22 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(82) CometFilter +(74) CometFilter Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(83) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(84) BroadcastExchange +(76) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 -BroadcastExchange (88) -+- * ColumnarToRow (87) - +- CometFilter (86) - +- CometScan parquet spark_catalog.default.date_dim (85) +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 +BroadcastExchange (80) ++- * ColumnarToRow (79) + +- CometFilter (78) + +- CometScan parquet spark_catalog.default.date_dim (77) (unknown) Scan parquet spark_catalog.default.date_dim @@ -496,19 +464,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(86) CometFilter +(78) CometFilter Input [2]: [d_date_sk#34, d_year#35] Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) -(87) ColumnarToRow [codegen id : 1] +(79) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] -(88) BroadcastExchange +(80) BroadcastExchange Input [2]: [d_date_sk#34, d_year#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#33 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#33 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index b1c026338c..562b5fdf29 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -8,11 +8,41 @@ TakeOrderedAndProject [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - RowToColumnar - WholeStageCodegen (3) + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -23,109 +53,71 @@ TakeOrderedAndProject [customer_preferred_cust_flag] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #6 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 WholeStageCodegen (11) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - RowToColumnar - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 5e524033e0..23ff8f948b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -1,28 +1,24 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * ColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- RowToColumnar (18) - +- * HashAggregate (17) - +- * ColumnarToRow (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (20) ++- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.web_sales @@ -68,7 +64,7 @@ Join condition: None Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 29] +(10) ReusedExchange [Reuses operator id: 25] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -88,57 +84,45 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) RowToColumnar +(14) Exchange Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] - -(17) HashAggregate [codegen id : 4] +(15) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] -(18) RowToColumnar -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] - -(19) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort +(16) Exchange Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6], [i_class#9 ASC NULLS FIRST] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) ColumnarToRow [codegen id : 5] +(17) Sort [codegen id : 5] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(22) Window +(18) Window Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(23) Project [codegen id : 6] +(19) Project [codegen id : 6] Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] -(24) TakeOrderedAndProject +(20) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (25) ++- * ColumnarToRow (24) + +- CometProject (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.date_dim (21) (unknown) Scan parquet spark_catalog.default.date_dim @@ -148,18 +132,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(26) CometFilter +(22) CometFilter Input [2]: [d_date_sk#11, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(27) CometProject +(23) CometProject Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(24) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(29) BroadcastExchange +(25) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index f1dcf3ef9f..fae1c6dba1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -4,41 +4,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (5) - ColumnarToRow + Sort [i_class] InputAdapter - CometSort [i_class] - CometColumnarExchange [i_class] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 + Exchange [i_class] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt index 61832c82d9..759871556e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt @@ -1,40 +1,38 @@ == Physical Plan == -* HashAggregate (36) -+- * ColumnarToRow (35) - +- CometColumnarExchange (34) - +- RowToColumnar (33) - +- * HashAggregate (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store (4) - : : : +- BroadcastExchange (14) - : : : +- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (17) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.customer_demographics (20) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.household_demographics (26) +* HashAggregate (34) ++- Exchange (33) + +- * HashAggregate (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store (4) + : : : +- BroadcastExchange (14) + : : : +- * ColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.customer_address (10) + : : +- ReusedExchange (17) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.customer_demographics (20) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.household_demographics (26) (unknown) Scan parquet spark_catalog.default.store_sales @@ -112,7 +110,7 @@ Join condition: ((((ca_state#14 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#14] -(17) ReusedExchange [Reuses operator id: 41] +(17) ReusedExchange [Reuses operator id: 39] Output [1]: [d_date_sk#16] (18) BroadcastHashJoin [codegen id : 6] @@ -188,17 +186,11 @@ Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sal Aggregate Attributes [7]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28] Results [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] -(33) RowToColumnar +(33) Exchange Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(34) CometColumnarExchange -Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(35) ColumnarToRow [codegen id : 7] -Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] - -(36) HashAggregate [codegen id : 7] +(34) HashAggregate [codegen id : 7] Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] @@ -208,11 +200,11 @@ Results [4]: [avg(ss_quantity#5)#36 AS avg(ss_quantity)#40, cast((avg(UnscaledVa ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (41) -+- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) +BroadcastExchange (39) ++- * ColumnarToRow (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.date_dim (35) (unknown) Scan parquet spark_catalog.default.date_dim @@ -222,18 +214,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(38) CometFilter +(36) CometFilter Input [2]: [d_date_sk#16, d_year#44] Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2001)) AND isnotnull(d_date_sk#16)) -(39) CometProject +(37) CometProject Input [2]: [d_date_sk#16, d_year#44] Arguments: [d_date_sk#16], [d_date_sk#16] -(40) ColumnarToRow [codegen id : 1] +(38) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] -(41) BroadcastExchange +(39) BroadcastExchange Input [1]: [d_date_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index 02184e3c7a..5e5fc41f83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -1,61 +1,59 @@ WholeStageCodegen (7) HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] - Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] - Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] - CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + Exchange #1 + WholeStageCodegen (6) + HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] + Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [hd_demo_sk,hd_dep_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index a57cf6fcfc..1323cb8df5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -1,119 +1,109 @@ == Physical Plan == -TakeOrderedAndProject (115) -+- * HashAggregate (114) - +- * ColumnarToRow (113) - +- CometColumnarExchange (112) - +- RowToColumnar (111) - +- * HashAggregate (110) - +- * Expand (109) - +- Union (108) - :- * Project (71) - : +- * Filter (70) - : +- * HashAggregate (69) - : +- * ColumnarToRow (68) - : +- CometColumnarExchange (67) - : +- RowToColumnar (66) - : +- * HashAggregate (65) - : +- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * BroadcastHashJoin LeftSemi BuildRight (53) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (52) - : : : +- * Project (51) - : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (49) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) - : : : :- * ColumnarToRow (37) - : : : : +- CometHashAggregate (36) - : : : : +- CometColumnarExchange (35) - : : : : +- RowToColumnar (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (47) - : : : +- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * Project (43) - : : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : : :- * ColumnarToRow (40) - : : : : : +- CometFilter (39) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) - : : : : +- ReusedExchange (41) - : : : +- ReusedExchange (44) - : : +- BroadcastExchange (59) - : : +- * BroadcastHashJoin LeftSemi BuildRight (58) - : : :- * ColumnarToRow (56) - : : : +- CometFilter (55) - : : : +- CometScan parquet spark_catalog.default.item (54) - : : +- ReusedExchange (57) - : +- ReusedExchange (62) - :- * Project (89) - : +- * Filter (88) - : +- * HashAggregate (87) - : +- * ColumnarToRow (86) - : +- CometColumnarExchange (85) - : +- RowToColumnar (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (79) - : : +- * BroadcastHashJoin Inner BuildRight (78) - : : :- * BroadcastHashJoin LeftSemi BuildRight (76) - : : : :- * ColumnarToRow (74) - : : : : +- CometFilter (73) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) - : : : +- ReusedExchange (75) - : : +- ReusedExchange (77) - : +- ReusedExchange (80) - +- * Project (107) - +- * Filter (106) - +- * HashAggregate (105) - +- * ColumnarToRow (104) - +- CometColumnarExchange (103) - +- RowToColumnar (102) - +- * HashAggregate (101) - +- * Project (100) - +- * BroadcastHashJoin Inner BuildRight (99) - :- * Project (97) - : +- * BroadcastHashJoin Inner BuildRight (96) - : :- * BroadcastHashJoin LeftSemi BuildRight (94) - : : :- * ColumnarToRow (92) - : : : +- CometFilter (91) - : : : +- CometScan parquet spark_catalog.default.web_sales (90) - : : +- ReusedExchange (93) - : +- ReusedExchange (95) - +- ReusedExchange (98) +TakeOrderedAndProject (105) ++- * HashAggregate (104) + +- Exchange (103) + +- * HashAggregate (102) + +- * Expand (101) + +- Union (100) + :- * Project (67) + : +- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : :- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * ColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * ColumnarToRow (38) + : : : : : +- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * ColumnarToRow (54) + : : : +- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + :- * Project (83) + : +- * Filter (82) + : +- * HashAggregate (81) + : +- Exchange (80) + : +- * HashAggregate (79) + : +- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * Project (75) + : : +- * BroadcastHashJoin Inner BuildRight (74) + : : :- * BroadcastHashJoin LeftSemi BuildRight (72) + : : : :- * ColumnarToRow (70) + : : : : +- CometFilter (69) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) + : : : +- ReusedExchange (71) + : : +- ReusedExchange (73) + : +- ReusedExchange (76) + +- * Project (99) + +- * Filter (98) + +- * HashAggregate (97) + +- Exchange (96) + +- * HashAggregate (95) + +- * Project (94) + +- * BroadcastHashJoin Inner BuildRight (93) + :- * Project (91) + : +- * BroadcastHashJoin Inner BuildRight (90) + : :- * BroadcastHashJoin LeftSemi BuildRight (88) + : : :- * ColumnarToRow (86) + : : : +- CometFilter (85) + : : : +- CometScan parquet spark_catalog.default.web_sales (84) + : : +- ReusedExchange (87) + : +- ReusedExchange (89) + +- ReusedExchange (92) (unknown) Scan parquet spark_catalog.default.store_sales @@ -217,7 +207,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 146] +(22) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -254,7 +244,7 @@ Join condition: None Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 146] +(30) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 6] @@ -274,20 +264,16 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) RowToColumnar +(34) Exchange Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) CometColumnarExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(36) CometHashAggregate +(35) HashAggregate [codegen id : 10] Input [3]: [brand_id#26, class_id#27, category_id#28] Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] - -(37) ColumnarToRow [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] (unknown) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] @@ -297,68 +283,68 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(39) CometFilter +(37) CometFilter Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] Condition : isnotnull(ws_item_sk#29) -(40) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 9] Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -(41) ReusedExchange [Reuses operator id: 19] +(39) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(42) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_item_sk#29] Right keys [1]: [i_item_sk#32] Join type: Inner Join condition: None -(43) Project [codegen id : 9] +(41) Project [codegen id : 9] Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(44) ReusedExchange [Reuses operator id: 146] +(42) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#36] -(45) BroadcastHashJoin [codegen id : 9] +(43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#30] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(46) Project [codegen id : 9] +(44) Project [codegen id : 9] Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] -(47) BroadcastExchange +(45) BroadcastExchange Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(48) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 10] Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] Join type: LeftSemi Join condition: None -(49) BroadcastExchange +(47) BroadcastExchange Input [3]: [brand_id#26, class_id#27, category_id#28] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(50) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#26, class_id#27, category_id#28] Join type: Inner Join condition: None -(51) Project [codegen id : 11] +(49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#37] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [ss_item_sk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(53) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi @@ -371,78 +357,72 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Condition : isnotnull(i_item_sk#38) -(56) ColumnarToRow [codegen id : 23] +(54) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(57) ReusedExchange [Reuses operator id: 52] +(55) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#37] -(58) BroadcastHashJoin [codegen id : 23] +(56) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#38] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(59) BroadcastExchange +(57) BroadcastExchange Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(61) Project [codegen id : 25] +(59) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(62) ReusedExchange [Reuses operator id: 141] +(60) ReusedExchange [Reuses operator id: 129] Output [1]: [d_date_sk#42] -(63) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#42] Join type: Inner Join condition: None -(64) Project [codegen id : 25] +(62) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(65) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(66) RowToColumnar -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] - -(67) CometColumnarExchange +(64) Exchange Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(68) ColumnarToRow [codegen id : 26] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] - -(69) HashAggregate [codegen id : 26] +(65) HashAggregate [codegen id : 26] Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] Results [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#51, count(1)#50 AS number_sales#52] -(70) Filter [codegen id : 26] +(66) Filter [codegen id : 26] Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) -(71) Project [codegen id : 26] +(67) Project [codegen id : 26] Output [6]: [sales#51, number_sales#52, store AS channel#55, i_brand_id#39 AS i_brand_id#56, i_class_id#40 AS i_class_id#57, i_category_id#41 AS i_category_id#58] Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] @@ -454,77 +434,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#62), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(73) CometFilter +(69) CometFilter Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] Condition : isnotnull(cs_item_sk#59) -(74) ColumnarToRow [codegen id : 51] +(70) ColumnarToRow [codegen id : 51] Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] -(75) ReusedExchange [Reuses operator id: 52] +(71) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#37] -(76) BroadcastHashJoin [codegen id : 51] +(72) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#59] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(77) ReusedExchange [Reuses operator id: 59] +(73) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] -(78) BroadcastHashJoin [codegen id : 51] +(74) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#59] Right keys [1]: [i_item_sk#64] Join type: Inner Join condition: None -(79) Project [codegen id : 51] +(75) Project [codegen id : 51] Output [6]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67] Input [8]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] -(80) ReusedExchange [Reuses operator id: 141] +(76) ReusedExchange [Reuses operator id: 129] Output [1]: [d_date_sk#68] -(81) BroadcastHashJoin [codegen id : 51] +(77) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_sold_date_sk#62] Right keys [1]: [d_date_sk#68] Join type: Inner Join condition: None -(82) Project [codegen id : 51] +(78) Project [codegen id : 51] Output [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] Input [7]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] -(83) HashAggregate [codegen id : 51] +(79) HashAggregate [codegen id : 51] Input [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] Functions [2]: [partial_sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), partial_count(1)] Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -(84) RowToColumnar -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] - -(85) CometColumnarExchange +(80) Exchange Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(86) ColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] - -(87) HashAggregate [codegen id : 52] +(81) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] Functions [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#75, count(1)#76] Results [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#75 AS sales#77, count(1)#76 AS number_sales#78] -(88) Filter [codegen id : 52] +(82) Filter [codegen id : 52] Input [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#77, number_sales#78] Condition : (isnotnull(sales#77) AND (cast(sales#77 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) -(89) Project [codegen id : 52] +(83) Project [codegen id : 52] Output [6]: [sales#77, number_sales#78, catalog AS channel#79, i_brand_id#65, i_class_id#66, i_category_id#67] Input [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#77, number_sales#78] @@ -536,138 +510,124 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(91) CometFilter +(85) CometFilter Input [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] Condition : isnotnull(ws_item_sk#80) -(92) ColumnarToRow [codegen id : 77] +(86) ColumnarToRow [codegen id : 77] Input [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] -(93) ReusedExchange [Reuses operator id: 52] +(87) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#37] -(94) BroadcastHashJoin [codegen id : 77] +(88) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#80] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(95) ReusedExchange [Reuses operator id: 59] +(89) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] -(96) BroadcastHashJoin [codegen id : 77] +(90) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#80] Right keys [1]: [i_item_sk#85] Join type: Inner Join condition: None -(97) Project [codegen id : 77] +(91) Project [codegen id : 77] Output [6]: [ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88] Input [8]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] -(98) ReusedExchange [Reuses operator id: 141] +(92) ReusedExchange [Reuses operator id: 129] Output [1]: [d_date_sk#89] -(99) BroadcastHashJoin [codegen id : 77] +(93) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_sold_date_sk#83] Right keys [1]: [d_date_sk#89] Join type: Inner Join condition: None -(100) Project [codegen id : 77] +(94) Project [codegen id : 77] Output [5]: [ws_quantity#81, ws_list_price#82, i_brand_id#86, i_class_id#87, i_category_id#88] Input [7]: [ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, d_date_sk#89] -(101) HashAggregate [codegen id : 77] +(95) HashAggregate [codegen id : 77] Input [5]: [ws_quantity#81, ws_list_price#82, i_brand_id#86, i_class_id#87, i_category_id#88] Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] Functions [2]: [partial_sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82)), partial_count(1)] Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] Results [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] -(102) RowToColumnar -Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] - -(103) CometColumnarExchange +(96) Exchange Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(104) ColumnarToRow [codegen id : 78] -Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] - -(105) HashAggregate [codegen id : 78] +(97) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] Functions [2]: [sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82))#96, count(1)#97] Results [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82))#96 AS sales#98, count(1)#97 AS number_sales#99] -(106) Filter [codegen id : 78] +(98) Filter [codegen id : 78] Input [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sales#98, number_sales#99] Condition : (isnotnull(sales#98) AND (cast(sales#98 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) -(107) Project [codegen id : 78] +(99) Project [codegen id : 78] Output [6]: [sales#98, number_sales#99, web AS channel#100, i_brand_id#86, i_class_id#87, i_category_id#88] Input [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sales#98, number_sales#99] -(108) Union +(100) Union -(109) Expand [codegen id : 79] +(101) Expand [codegen id : 79] Input [6]: [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58] Arguments: [[sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58, 0], [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, null, 1], [sales#51, number_sales#52, channel#55, i_brand_id#56, null, null, 3], [sales#51, number_sales#52, channel#55, null, null, null, 7], [sales#51, number_sales#52, null, null, null, null, 15]], [sales#51, number_sales#52, channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] -(110) HashAggregate [codegen id : 79] +(102) HashAggregate [codegen id : 79] Input [7]: [sales#51, number_sales#52, channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] Keys [5]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] Functions [2]: [partial_sum(sales#51), partial_sum(number_sales#52)] Aggregate Attributes [3]: [sum#106, isEmpty#107, sum#108] Results [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] -(111) RowToColumnar -Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] - -(112) CometColumnarExchange -Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] -Arguments: hashpartitioning(channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(113) ColumnarToRow [codegen id : 80] +(103) Exchange Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] +Arguments: hashpartitioning(channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(114) HashAggregate [codegen id : 80] +(104) HashAggregate [codegen id : 80] Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] Keys [5]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] Functions [2]: [sum(sales#51), sum(number_sales#52)] Aggregate Attributes [2]: [sum(sales#51)#112, sum(number_sales#52)#113] Results [6]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales#51)#112 AS sum(sales)#114, sum(number_sales#52)#113 AS sum(number_sales)#115] -(115) TakeOrderedAndProject +(105) TakeOrderedAndProject Input [6]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales)#114, sum(number_sales)#115] Arguments: 100, [channel#101 ASC NULLS FIRST, i_brand_id#102 ASC NULLS FIRST, i_class_id#103 ASC NULLS FIRST, i_category_id#104 ASC NULLS FIRST], [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales)#114, sum(number_sales)#115] ===== Subqueries ===== -Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* HashAggregate (136) -+- * ColumnarToRow (135) - +- CometColumnarExchange (134) - +- RowToColumnar (133) - +- * HashAggregate (132) - +- Union (131) - :- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * ColumnarToRow (117) - : : +- CometScan parquet spark_catalog.default.store_sales (116) - : +- ReusedExchange (118) - :- * Project (125) - : +- * BroadcastHashJoin Inner BuildRight (124) - : :- * ColumnarToRow (122) - : : +- CometScan parquet spark_catalog.default.catalog_sales (121) - : +- ReusedExchange (123) - +- * Project (130) - +- * BroadcastHashJoin Inner BuildRight (129) - :- * ColumnarToRow (127) - : +- CometScan parquet spark_catalog.default.web_sales (126) - +- ReusedExchange (128) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#53, [id=#54] +* HashAggregate (124) ++- Exchange (123) + +- * HashAggregate (122) + +- Union (121) + :- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * ColumnarToRow (107) + : : +- CometScan parquet spark_catalog.default.store_sales (106) + : +- ReusedExchange (108) + :- * Project (115) + : +- * BroadcastHashJoin Inner BuildRight (114) + : :- * ColumnarToRow (112) + : : +- CometScan parquet spark_catalog.default.catalog_sales (111) + : +- ReusedExchange (113) + +- * Project (120) + +- * BroadcastHashJoin Inner BuildRight (119) + :- * ColumnarToRow (117) + : +- CometScan parquet spark_catalog.default.web_sales (116) + +- ReusedExchange (118) (unknown) Scan parquet spark_catalog.default.store_sales @@ -677,19 +637,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#118), dynamicpruningexpression(ss_sold_date_sk#118 IN dynamicpruning#119)] ReadSchema: struct -(117) ColumnarToRow [codegen id : 2] +(107) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#116, ss_list_price#117, ss_sold_date_sk#118] -(118) ReusedExchange [Reuses operator id: 146] +(108) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#120] -(119) BroadcastHashJoin [codegen id : 2] +(109) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#118] Right keys [1]: [d_date_sk#120] Join type: Inner Join condition: None -(120) Project [codegen id : 2] +(110) Project [codegen id : 2] Output [2]: [ss_quantity#116 AS quantity#121, ss_list_price#117 AS list_price#122] Input [4]: [ss_quantity#116, ss_list_price#117, ss_sold_date_sk#118, d_date_sk#120] @@ -700,19 +660,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#125), dynamicpruningexpression(cs_sold_date_sk#125 IN dynamicpruning#126)] ReadSchema: struct -(122) ColumnarToRow [codegen id : 4] +(112) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#123, cs_list_price#124, cs_sold_date_sk#125] -(123) ReusedExchange [Reuses operator id: 146] +(113) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#127] -(124) BroadcastHashJoin [codegen id : 4] +(114) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#125] Right keys [1]: [d_date_sk#127] Join type: Inner Join condition: None -(125) Project [codegen id : 4] +(115) Project [codegen id : 4] Output [2]: [cs_quantity#123 AS quantity#128, cs_list_price#124 AS list_price#129] Input [4]: [cs_quantity#123, cs_list_price#124, cs_sold_date_sk#125, d_date_sk#127] @@ -723,60 +683,54 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#132), dynamicpruningexpression(ws_sold_date_sk#132 IN dynamicpruning#133)] ReadSchema: struct -(127) ColumnarToRow [codegen id : 6] +(117) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#130, ws_list_price#131, ws_sold_date_sk#132] -(128) ReusedExchange [Reuses operator id: 146] +(118) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#134] -(129) BroadcastHashJoin [codegen id : 6] +(119) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#132] Right keys [1]: [d_date_sk#134] Join type: Inner Join condition: None -(130) Project [codegen id : 6] +(120) Project [codegen id : 6] Output [2]: [ws_quantity#130 AS quantity#135, ws_list_price#131 AS list_price#136] Input [4]: [ws_quantity#130, ws_list_price#131, ws_sold_date_sk#132, d_date_sk#134] -(131) Union +(121) Union -(132) HashAggregate [codegen id : 7] +(122) HashAggregate [codegen id : 7] Input [2]: [quantity#121, list_price#122] Keys: [] Functions [1]: [partial_avg((cast(quantity#121 as decimal(10,0)) * list_price#122))] Aggregate Attributes [2]: [sum#137, count#138] Results [2]: [sum#139, count#140] -(133) RowToColumnar -Input [2]: [sum#139, count#140] - -(134) CometColumnarExchange -Input [2]: [sum#139, count#140] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(135) ColumnarToRow [codegen id : 8] +(123) Exchange Input [2]: [sum#139, count#140] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(136) HashAggregate [codegen id : 8] +(124) HashAggregate [codegen id : 8] Input [2]: [sum#139, count#140] Keys: [] Functions [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))] Aggregate Attributes [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))#141] Results [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))#141 AS average_sales#142] -Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#118 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#118 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 121 Hosting Expression = cs_sold_date_sk#125 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 111 Hosting Expression = cs_sold_date_sk#125 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 126 Hosting Expression = ws_sold_date_sk#132 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 116 Hosting Expression = ws_sold_date_sk#132 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (141) -+- * ColumnarToRow (140) - +- CometProject (139) - +- CometFilter (138) - +- CometScan parquet spark_catalog.default.date_dim (137) +BroadcastExchange (129) ++- * ColumnarToRow (128) + +- CometProject (127) + +- CometFilter (126) + +- CometScan parquet spark_catalog.default.date_dim (125) (unknown) Scan parquet spark_catalog.default.date_dim @@ -786,27 +740,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(138) CometFilter +(126) CometFilter Input [3]: [d_date_sk#42, d_year#143, d_moy#144] Condition : ((((isnotnull(d_year#143) AND isnotnull(d_moy#144)) AND (d_year#143 = 2001)) AND (d_moy#144 = 11)) AND isnotnull(d_date_sk#42)) -(139) CometProject +(127) CometProject Input [3]: [d_date_sk#42, d_year#143, d_moy#144] Arguments: [d_date_sk#42], [d_date_sk#42] -(140) ColumnarToRow [codegen id : 1] +(128) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#42] -(141) BroadcastExchange +(129) BroadcastExchange Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (146) -+- * ColumnarToRow (145) - +- CometProject (144) - +- CometFilter (143) - +- CometScan parquet spark_catalog.default.date_dim (142) +BroadcastExchange (134) ++- * ColumnarToRow (133) + +- CometProject (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) (unknown) Scan parquet spark_catalog.default.date_dim @@ -816,31 +770,31 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(143) CometFilter +(131) CometFilter Input [2]: [d_date_sk#25, d_year#145] Condition : (((isnotnull(d_year#145) AND (d_year#145 >= 1999)) AND (d_year#145 <= 2001)) AND isnotnull(d_date_sk#25)) -(144) CometProject +(132) CometProject Input [2]: [d_date_sk#25, d_year#145] Arguments: [d_date_sk#25], [d_date_sk#25] -(145) ColumnarToRow [codegen id : 1] +(133) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(146) BroadcastExchange +(134) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 88 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] -Subquery:10 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] -Subquery:12 Hosting operator id = 90 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index 8d20005f9e..cf688c4486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -1,226 +1,214 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] WholeStageCodegen (80) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - RowToColumnar - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - InputAdapter - Union - WholeStageCodegen (26) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #13 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 + WholeStageCodegen (79) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] + InputAdapter + Union + WholeStageCodegen (26) + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] + Filter [sales] + Subquery #3 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + InputAdapter + Exchange #13 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #7 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #7 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #7 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #2 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #7 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 - RowToColumnar - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - ColumnarToRow + BroadcastExchange #5 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #6 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + BroadcastExchange #10 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (52) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - RowToColumnar - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk] #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (52) + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (51) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (78) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #15 - RowToColumnar - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (78) + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (77) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 2c7c8b099e..536306de5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -1,94 +1,88 @@ == Physical Plan == -TakeOrderedAndProject (90) -+- * BroadcastHashJoin Inner BuildRight (89) - :- * Filter (70) - : +- * HashAggregate (69) - : +- * ColumnarToRow (68) - : +- CometColumnarExchange (67) - : +- RowToColumnar (66) - : +- * HashAggregate (65) - : +- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * BroadcastHashJoin LeftSemi BuildRight (53) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (52) - : : : +- * Project (51) - : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (49) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) - : : : :- * ColumnarToRow (37) - : : : : +- CometHashAggregate (36) - : : : : +- CometColumnarExchange (35) - : : : : +- RowToColumnar (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (47) - : : : +- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * Project (43) - : : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : : :- * ColumnarToRow (40) - : : : : : +- CometFilter (39) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) - : : : : +- ReusedExchange (41) - : : : +- ReusedExchange (44) - : : +- BroadcastExchange (59) - : : +- * BroadcastHashJoin LeftSemi BuildRight (58) - : : :- * ColumnarToRow (56) - : : : +- CometFilter (55) - : : : +- CometScan parquet spark_catalog.default.item (54) - : : +- ReusedExchange (57) - : +- ReusedExchange (62) - +- BroadcastExchange (88) - +- * Filter (87) - +- * HashAggregate (86) - +- * ColumnarToRow (85) - +- CometColumnarExchange (84) - +- RowToColumnar (83) - +- * HashAggregate (82) - +- * Project (81) - +- * BroadcastHashJoin Inner BuildRight (80) - :- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * BroadcastHashJoin LeftSemi BuildRight (75) - : : :- * ColumnarToRow (73) - : : : +- CometFilter (72) - : : : +- CometScan parquet spark_catalog.default.store_sales (71) - : : +- ReusedExchange (74) - : +- ReusedExchange (76) - +- ReusedExchange (79) +TakeOrderedAndProject (84) ++- * BroadcastHashJoin Inner BuildRight (83) + :- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : :- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * ColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * ColumnarToRow (38) + : : : : : +- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * ColumnarToRow (54) + : : : +- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + +- BroadcastExchange (82) + +- * Filter (81) + +- * HashAggregate (80) + +- Exchange (79) + +- * HashAggregate (78) + +- * Project (77) + +- * BroadcastHashJoin Inner BuildRight (76) + :- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : :- * ColumnarToRow (69) + : : : +- CometFilter (68) + : : : +- CometScan parquet spark_catalog.default.store_sales (67) + : : +- ReusedExchange (70) + : +- ReusedExchange (72) + +- ReusedExchange (75) (unknown) Scan parquet spark_catalog.default.store_sales @@ -192,7 +186,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 125] +(22) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -229,7 +223,7 @@ Join condition: None Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 125] +(30) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 6] @@ -249,20 +243,16 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) RowToColumnar +(34) Exchange Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) CometColumnarExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(36) CometHashAggregate +(35) HashAggregate [codegen id : 10] Input [3]: [brand_id#26, class_id#27, category_id#28] Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] - -(37) ColumnarToRow [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] (unknown) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] @@ -272,68 +262,68 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(39) CometFilter +(37) CometFilter Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] Condition : isnotnull(ws_item_sk#29) -(40) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 9] Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -(41) ReusedExchange [Reuses operator id: 19] +(39) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(42) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_item_sk#29] Right keys [1]: [i_item_sk#32] Join type: Inner Join condition: None -(43) Project [codegen id : 9] +(41) Project [codegen id : 9] Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(44) ReusedExchange [Reuses operator id: 125] +(42) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#36] -(45) BroadcastHashJoin [codegen id : 9] +(43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#30] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(46) Project [codegen id : 9] +(44) Project [codegen id : 9] Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] -(47) BroadcastExchange +(45) BroadcastExchange Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(48) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 10] Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] Join type: LeftSemi Join condition: None -(49) BroadcastExchange +(47) BroadcastExchange Input [3]: [brand_id#26, class_id#27, category_id#28] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(50) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#26, class_id#27, category_id#28] Join type: Inner Join condition: None -(51) Project [codegen id : 11] +(49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#37] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [ss_item_sk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(53) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi @@ -346,74 +336,68 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) -(56) ColumnarToRow [codegen id : 23] +(54) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(57) ReusedExchange [Reuses operator id: 52] +(55) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#37] -(58) BroadcastHashJoin [codegen id : 23] +(56) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#38] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(59) BroadcastExchange +(57) BroadcastExchange Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(61) Project [codegen id : 25] +(59) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(62) ReusedExchange [Reuses operator id: 116] +(60) ReusedExchange [Reuses operator id: 108] Output [1]: [d_date_sk#42] -(63) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#42] Join type: Inner Join condition: None -(64) Project [codegen id : 25] +(62) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(65) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(66) RowToColumnar -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] - -(67) CometColumnarExchange +(64) Exchange Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(68) ColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] - -(69) HashAggregate [codegen id : 52] +(65) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(70) Filter [codegen id : 52] +(66) Filter [codegen id : 52] Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) @@ -425,114 +409,106 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(72) CometFilter +(68) CometFilter Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] Condition : isnotnull(ss_item_sk#56) -(73) ColumnarToRow [codegen id : 50] +(69) ColumnarToRow [codegen id : 50] Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] -(74) ReusedExchange [Reuses operator id: 52] +(70) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#37] -(75) BroadcastHashJoin [codegen id : 50] +(71) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#56] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(76) ReusedExchange [Reuses operator id: 59] +(72) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -(77) BroadcastHashJoin [codegen id : 50] +(73) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#56] Right keys [1]: [i_item_sk#61] Join type: Inner Join condition: None -(78) Project [codegen id : 50] +(74) Project [codegen id : 50] Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -(79) ReusedExchange [Reuses operator id: 130] +(75) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#65] -(80) BroadcastHashJoin [codegen id : 50] +(76) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_sold_date_sk#59] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(81) Project [codegen id : 50] +(77) Project [codegen id : 50] Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -(82) HashAggregate [codegen id : 50] +(78) HashAggregate [codegen id : 50] Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -(83) RowToColumnar -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] - -(84) CometColumnarExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(85) ColumnarToRow [codegen id : 51] +(79) Exchange Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(86) HashAggregate [codegen id : 51] +(80) HashAggregate [codegen id : 51] Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72, count(1)#73] Results [6]: [store AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] -(87) Filter [codegen id : 51] +(81) Filter [codegen id : 51] Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(88) BroadcastExchange +(82) BroadcastExchange Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(89) BroadcastHashJoin [codegen id : 52] +(83) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Right keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Join type: Inner Join condition: None -(90) TakeOrderedAndProject +(84) TakeOrderedAndProject Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] ===== Subqueries ===== -Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (111) -+- * ColumnarToRow (110) - +- CometColumnarExchange (109) - +- RowToColumnar (108) - +- * HashAggregate (107) - +- Union (106) - :- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * ColumnarToRow (92) - : : +- CometScan parquet spark_catalog.default.store_sales (91) - : +- ReusedExchange (93) - :- * Project (100) - : +- * BroadcastHashJoin Inner BuildRight (99) - : :- * ColumnarToRow (97) - : : +- CometScan parquet spark_catalog.default.catalog_sales (96) - : +- ReusedExchange (98) - +- * Project (105) - +- * BroadcastHashJoin Inner BuildRight (104) - :- * ColumnarToRow (102) - : +- CometScan parquet spark_catalog.default.web_sales (101) - +- ReusedExchange (103) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (103) ++- Exchange (102) + +- * HashAggregate (101) + +- Union (100) + :- * Project (89) + : +- * BroadcastHashJoin Inner BuildRight (88) + : :- * ColumnarToRow (86) + : : +- CometScan parquet spark_catalog.default.store_sales (85) + : +- ReusedExchange (87) + :- * Project (94) + : +- * BroadcastHashJoin Inner BuildRight (93) + : :- * ColumnarToRow (91) + : : +- CometScan parquet spark_catalog.default.catalog_sales (90) + : +- ReusedExchange (92) + +- * Project (99) + +- * BroadcastHashJoin Inner BuildRight (98) + :- * ColumnarToRow (96) + : +- CometScan parquet spark_catalog.default.web_sales (95) + +- ReusedExchange (97) (unknown) Scan parquet spark_catalog.default.store_sales @@ -542,19 +518,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#79), dynamicpruningexpression(ss_sold_date_sk#79 IN dynamicpruning#80)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 2] +(86) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] -(93) ReusedExchange [Reuses operator id: 125] +(87) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#81] -(94) BroadcastHashJoin [codegen id : 2] +(88) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#79] Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(95) Project [codegen id : 2] +(89) Project [codegen id : 2] Output [2]: [ss_quantity#77 AS quantity#82, ss_list_price#78 AS list_price#83] Input [4]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79, d_date_sk#81] @@ -565,19 +541,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#86), dynamicpruningexpression(cs_sold_date_sk#86 IN dynamicpruning#87)] ReadSchema: struct -(97) ColumnarToRow [codegen id : 4] +(91) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] -(98) ReusedExchange [Reuses operator id: 125] +(92) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#88] -(99) BroadcastHashJoin [codegen id : 4] +(93) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#86] Right keys [1]: [d_date_sk#88] Join type: Inner Join condition: None -(100) Project [codegen id : 4] +(94) Project [codegen id : 4] Output [2]: [cs_quantity#84 AS quantity#89, cs_list_price#85 AS list_price#90] Input [4]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86, d_date_sk#88] @@ -588,60 +564,54 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 6] +(96) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] -(103) ReusedExchange [Reuses operator id: 125] +(97) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#95] -(104) BroadcastHashJoin [codegen id : 6] +(98) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#93] Right keys [1]: [d_date_sk#95] Join type: Inner Join condition: None -(105) Project [codegen id : 6] +(99) Project [codegen id : 6] Output [2]: [ws_quantity#91 AS quantity#96, ws_list_price#92 AS list_price#97] Input [4]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93, d_date_sk#95] -(106) Union +(100) Union -(107) HashAggregate [codegen id : 7] +(101) HashAggregate [codegen id : 7] Input [2]: [quantity#82, list_price#83] Keys: [] Functions [1]: [partial_avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] Aggregate Attributes [2]: [sum#98, count#99] Results [2]: [sum#100, count#101] -(108) RowToColumnar -Input [2]: [sum#100, count#101] - -(109) CometColumnarExchange -Input [2]: [sum#100, count#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(110) ColumnarToRow [codegen id : 8] +(102) Exchange Input [2]: [sum#100, count#101] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(111) HashAggregate [codegen id : 8] +(103) HashAggregate [codegen id : 8] Input [2]: [sum#100, count#101] Keys: [] Functions [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] Aggregate Attributes [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102] Results [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102 AS average_sales#103] -Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#79 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#79 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (116) -+- * ColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan parquet spark_catalog.default.date_dim (112) +BroadcastExchange (108) ++- * ColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) (unknown) Scan parquet spark_catalog.default.date_dim @@ -651,26 +621,26 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(113) CometFilter +(105) CometFilter Input [2]: [d_date_sk#42, d_week_seq#104] Condition : ((isnotnull(d_week_seq#104) AND (d_week_seq#104 = Subquery scalar-subquery#105, [id=#106])) AND isnotnull(d_date_sk#42)) -(114) CometProject +(106) CometProject Input [2]: [d_date_sk#42, d_week_seq#104] Arguments: [d_date_sk#42], [d_date_sk#42] -(115) ColumnarToRow [codegen id : 1] +(107) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#42] -(116) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 113 Hosting Expression = Subquery scalar-subquery#105, [id=#106] -* ColumnarToRow (120) -+- CometProject (119) - +- CometFilter (118) - +- CometScan parquet spark_catalog.default.date_dim (117) +Subquery:6 Hosting operator id = 105 Hosting Expression = Subquery scalar-subquery#105, [id=#106] +* ColumnarToRow (112) ++- CometProject (111) + +- CometFilter (110) + +- CometScan parquet spark_catalog.default.date_dim (109) (unknown) Scan parquet spark_catalog.default.date_dim @@ -680,23 +650,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(118) CometFilter +(110) CometFilter Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] Condition : (((((isnotnull(d_year#108) AND isnotnull(d_moy#109)) AND isnotnull(d_dom#110)) AND (d_year#108 = 2000)) AND (d_moy#109 = 12)) AND (d_dom#110 = 11)) -(119) CometProject +(111) CometProject Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] Arguments: [d_week_seq#107], [d_week_seq#107] -(120) ColumnarToRow [codegen id : 1] +(112) ColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#107] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (125) -+- * ColumnarToRow (124) - +- CometProject (123) - +- CometFilter (122) - +- CometScan parquet spark_catalog.default.date_dim (121) +BroadcastExchange (117) ++- * ColumnarToRow (116) + +- CometProject (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) (unknown) Scan parquet spark_catalog.default.date_dim @@ -706,33 +676,33 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(122) CometFilter +(114) CometFilter Input [2]: [d_date_sk#25, d_year#111] Condition : (((isnotnull(d_year#111) AND (d_year#111 >= 1999)) AND (d_year#111 <= 2001)) AND isnotnull(d_date_sk#25)) -(123) CometProject +(115) CometProject Input [2]: [d_date_sk#25, d_year#111] Arguments: [d_date_sk#25], [d_date_sk#25] -(124) ColumnarToRow [codegen id : 1] +(116) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(125) BroadcastExchange +(117) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 -BroadcastExchange (130) -+- * ColumnarToRow (129) - +- CometProject (128) - +- CometFilter (127) - +- CometScan parquet spark_catalog.default.date_dim (126) +Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 +BroadcastExchange (122) ++- * ColumnarToRow (121) + +- CometProject (120) + +- CometFilter (119) + +- CometScan parquet spark_catalog.default.date_dim (118) (unknown) Scan parquet spark_catalog.default.date_dim @@ -742,26 +712,26 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(127) CometFilter +(119) CometFilter Input [2]: [d_date_sk#65, d_week_seq#112] Condition : ((isnotnull(d_week_seq#112) AND (d_week_seq#112 = Subquery scalar-subquery#113, [id=#114])) AND isnotnull(d_date_sk#65)) -(128) CometProject +(120) CometProject Input [2]: [d_date_sk#65, d_week_seq#112] Arguments: [d_date_sk#65], [d_date_sk#65] -(129) ColumnarToRow [codegen id : 1] +(121) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#65] -(130) BroadcastExchange +(122) BroadcastExchange Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:12 Hosting operator id = 127 Hosting Expression = Subquery scalar-subquery#113, [id=#114] -* ColumnarToRow (134) -+- CometProject (133) - +- CometFilter (132) - +- CometScan parquet spark_catalog.default.date_dim (131) +Subquery:12 Hosting operator id = 119 Hosting Expression = Subquery scalar-subquery#113, [id=#114] +* ColumnarToRow (126) ++- CometProject (125) + +- CometFilter (124) + +- CometScan parquet spark_catalog.default.date_dim (123) (unknown) Scan parquet spark_catalog.default.date_dim @@ -771,15 +741,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(132) CometFilter +(124) CometFilter Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] Condition : (((((isnotnull(d_year#116) AND isnotnull(d_moy#117)) AND isnotnull(d_dom#118)) AND (d_year#116 = 1999)) AND (d_moy#117 = 12)) AND (d_dom#118 = 11)) -(133) CometProject +(125) CometProject Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] Arguments: [d_week_seq#115], [d_week_seq#115] -(134) ColumnarToRow [codegen id : 1] +(126) ColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#115] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 29d5a8d87c..09d8d9dde3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -5,206 +5,198 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Subquery #4 WholeStageCodegen (8) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #12 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - RowToColumnar - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] + InputAdapter + Exchange #12 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #5 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ColumnarToRow + InputAdapter + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #11 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 WholeStageCodegen (51) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - RowToColumnar - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] - Subquery #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (50) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + BroadcastExchange #15 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + Subquery #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + ReusedExchange [ss_item_sk] #3 InputAdapter - ReusedExchange [d_date_sk] #15 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt index 15682a488b..811a9eef51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt @@ -1,28 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- * ColumnarToRow (22) - +- CometColumnarExchange (21) - +- RowToColumnar (20) - +- * HashAggregate (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.customer (4) - : +- BroadcastExchange (13) - : +- * ColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.customer_address (10) - +- ReusedExchange (16) +TakeOrderedAndProject (22) ++- * HashAggregate (21) + +- Exchange (20) + +- * HashAggregate (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (15) + : +- * BroadcastHashJoin Inner BuildRight (14) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.customer (4) + : +- BroadcastExchange (13) + : +- * ColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.customer_address (10) + +- ReusedExchange (16) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -96,7 +94,7 @@ Join condition: ((substr(ca_zip#9, 1, 5) IN (85669,86197,88274,83405,86475,85392 Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9] Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#8, ca_zip#9] -(16) ReusedExchange [Reuses operator id: 29] +(16) ReusedExchange [Reuses operator id: 27] Output [1]: [d_date_sk#10] (17) BroadcastHashJoin [codegen id : 4] @@ -116,35 +114,29 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] Aggregate Attributes [1]: [sum#11] Results [2]: [ca_zip#9, sum#12] -(20) RowToColumnar +(20) Exchange Input [2]: [ca_zip#9, sum#12] +Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometColumnarExchange -Input [2]: [ca_zip#9, sum#12] -Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) ColumnarToRow [codegen id : 5] -Input [2]: [ca_zip#9, sum#12] - -(23) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 5] Input [2]: [ca_zip#9, sum#12] Keys [1]: [ca_zip#9] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#13] Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#13,17,2) AS sum(cs_sales_price)#14] -(24) TakeOrderedAndProject +(22) TakeOrderedAndProject Input [2]: [ca_zip#9, sum(cs_sales_price)#14] Arguments: 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) (unknown) Scan parquet spark_catalog.default.date_dim @@ -154,18 +146,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(26) CometFilter +(24) CometFilter Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] Condition : ((((isnotnull(d_qoy#16) AND isnotnull(d_year#15)) AND (d_qoy#16 = 2)) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#10)) -(27) CometProject +(25) CometProject Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] Arguments: [d_date_sk#10], [d_date_sk#10] -(28) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(29) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index 96fe9a3d78..5c750b2db8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -1,43 +1,41 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (5) HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [ca_zip,cs_sales_price] [sum,sum] - Project [cs_sales_price,ca_zip] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] - Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + Exchange [ca_zip] #1 + WholeStageCodegen (4) + HashAggregate [ca_zip,cs_sales_price] [sum,sum] + Project [cs_sales_price,ca_zip] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sales_price,cs_sold_date_sk,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] + InputAdapter + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index c02e9680b6..d8ed9c06ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -1,51 +1,49 @@ == Physical Plan == -* HashAggregate (47) -+- * ColumnarToRow (46) - +- CometColumnarExchange (45) - +- RowToColumnar (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometSort (5) - : : : : : +- CometColumnarExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- * ColumnarToRow (11) - : : : : +- CometSort (10) - : : : : +- CometColumnarExchange (9) - : : : : +- CometProject (8) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) - : : : +- * ColumnarToRow (18) - : : : +- CometSort (17) - : : : +- CometColumnarExchange (16) - : : : +- CometProject (15) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.call_center (34) +* HashAggregate (45) ++- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- * SortMergeJoin LeftSemi (12) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * ColumnarToRow (9) + : : : : +- CometProject (8) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * ColumnarToRow (16) + : : : +- CometProject (15) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometProject (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.date_dim (20) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.call_center (34) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -63,16 +61,16 @@ Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(4) CometColumnarExchange +(4) ColumnarToRow [codegen id : 1] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) CometSort +(5) Exchange Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] +Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(6) ColumnarToRow [codegen id : 1] +(6) Sort [codegen id : 2] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] @@ -84,24 +82,24 @@ ReadSchema: struct Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] -(9) CometColumnarExchange +(9) ColumnarToRow [codegen id : 3] Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) CometSort +(10) Exchange Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] +Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) ColumnarToRow [codegen id : 2] +(11) Sort [codegen id : 4] Input [2]: [cs_warehouse_sk#9, cs_order_number#10] +Arguments: [cs_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 3] +(12) SortMergeJoin [codegen id : 5] Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#10] Join type: LeftSemi Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) -(13) Project [codegen id : 3] +(13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -115,18 +113,18 @@ ReadSchema: struct Input [2]: [cr_order_number#12, cr_returned_date_sk#13] Arguments: [cr_order_number#12], [cr_order_number#12] -(16) CometColumnarExchange +(16) ColumnarToRow [codegen id : 6] Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) CometSort +(17) Exchange Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(18) ColumnarToRow [codegen id : 4] +(18) Sort [codegen id : 7] Input [1]: [cr_order_number#12] +Arguments: [cr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 8] +(19) SortMergeJoin [codegen id : 11] Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#12] Join type: LeftAnti @@ -147,20 +145,20 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(23) ColumnarToRow [codegen id : 5] +(23) ColumnarToRow [codegen id : 8] Input [1]: [d_date_sk#14] (24) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 8] +(25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 8] +(26) Project [codegen id : 11] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] @@ -179,20 +177,20 @@ Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_ad Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) ColumnarToRow [codegen id : 6] +(30) ColumnarToRow [codegen id : 9] Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(33) Project [codegen id : 11] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] @@ -211,55 +209,49 @@ Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AN Input [2]: [cc_call_center_sk#18, cc_county#19] Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] -(37) ColumnarToRow [codegen id : 7] +(37) ColumnarToRow [codegen id : 10] Input [1]: [cc_call_center_sk#18] (38) BroadcastExchange Input [1]: [cc_call_center_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 8] +(40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -(41) HashAggregate [codegen id : 8] +(41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 8] +(42) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 8] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) RowToColumnar +(44) Exchange Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) CometColumnarExchange -Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(46) ColumnarToRow [codegen id : 9] -Input [3]: [sum#22, sum#23, count#25] - -(47) HashAggregate [codegen id : 9] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index 97ea38304d..a55c182bed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -1,70 +1,74 @@ -WholeStageCodegen (9) +WholeStageCodegen (12) HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] - Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - SortMergeJoin [cs_order_number,cr_order_number] - InputAdapter - WholeStageCodegen (3) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - InputAdapter - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometSort [cs_order_number] - CometColumnarExchange [cs_order_number] #2 + InputAdapter + Exchange #1 + WholeStageCodegen (11) + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] + Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + SortMergeJoin [cs_order_number,cr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_order_number] + InputAdapter + Exchange [cs_order_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometSort [cs_order_number] - CometColumnarExchange [cs_order_number] #3 + InputAdapter + WholeStageCodegen (4) + Sort [cs_order_number] + InputAdapter + Exchange [cs_order_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter CometProject [cs_warehouse_sk,cs_order_number] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometSort [cr_order_number] - CometColumnarExchange [cr_order_number] #4 + InputAdapter + WholeStageCodegen (7) + Sort [cr_order_number] + InputAdapter + Exchange [cr_order_number] #4 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter CometProject [cr_order_number] CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #5 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (9) ColumnarToRow InputAdapter - CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometProject [cc_call_center_sk] + CometFilter [cc_county,cc_call_center_sk] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index 759cb2384e..5e9b6d1da7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -1,46 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- * ColumnarToRow (40) - +- CometColumnarExchange (39) - +- RowToColumnar (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +TakeOrderedAndProject (40) ++- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (18) + : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) + : : : : +- ReusedExchange (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.store (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.item (31) (unknown) Scan parquet spark_catalog.default.store_sales @@ -116,7 +114,7 @@ Join condition: None Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -(16) ReusedExchange [Reuses operator id: 47] +(16) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#19] (17) BroadcastHashJoin [codegen id : 8] @@ -129,7 +127,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -(19) ReusedExchange [Reuses operator id: 52] +(19) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#20] (20) BroadcastHashJoin [codegen id : 8] @@ -142,7 +140,7 @@ Join condition: None Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] -(22) ReusedExchange [Reuses operator id: 52] +(22) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#21] (23) BroadcastHashJoin [codegen id : 8] @@ -218,35 +216,29 @@ Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partia Aggregate Attributes [18]: [count#27, sum#28, count#29, n#30, avg#31, m2#32, count#33, sum#34, count#35, n#36, avg#37, m2#38, count#39, sum#40, count#41, n#42, avg#43, m2#44] Results [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] -(38) RowToColumnar +(38) Exchange Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] +Arguments: hashpartitioning(i_item_id#25, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(39) CometColumnarExchange -Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] -Arguments: hashpartitioning(i_item_id#25, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) ColumnarToRow [codegen id : 9] -Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] - -(41) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 9] Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] Keys [3]: [i_item_id#25, i_item_desc#26, s_state#23] Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] Aggregate Attributes [9]: [count(ss_quantity#5)#63, avg(ss_quantity#5)#64, stddev_samp(cast(ss_quantity#5 as double))#65, count(sr_return_quantity#11)#66, avg(sr_return_quantity#11)#67, stddev_samp(cast(sr_return_quantity#11 as double))#68, count(cs_quantity#16)#69, avg(cs_quantity#16)#70, stddev_samp(cast(cs_quantity#16 as double))#71] Results [15]: [i_item_id#25, i_item_desc#26, s_state#23, count(ss_quantity#5)#63 AS store_sales_quantitycount#72, avg(ss_quantity#5)#64 AS store_sales_quantityave#73, stddev_samp(cast(ss_quantity#5 as double))#65 AS store_sales_quantitystdev#74, (stddev_samp(cast(ss_quantity#5 as double))#65 / avg(ss_quantity#5)#64) AS store_sales_quantitycov#75, count(sr_return_quantity#11)#66 AS as_store_returns_quantitycount#76, avg(sr_return_quantity#11)#67 AS as_store_returns_quantityave#77, stddev_samp(cast(sr_return_quantity#11 as double))#68 AS as_store_returns_quantitystdev#78, (stddev_samp(cast(sr_return_quantity#11 as double))#68 / avg(sr_return_quantity#11)#67) AS store_returns_quantitycov#79, count(cs_quantity#16)#69 AS catalog_sales_quantitycount#80, avg(cs_quantity#16)#70 AS catalog_sales_quantityave#81, (stddev_samp(cast(cs_quantity#16 as double))#71 / avg(cs_quantity#16)#70) AS catalog_sales_quantitystdev#82, (stddev_samp(cast(cs_quantity#16 as double))#71 / avg(cs_quantity#16)#70) AS catalog_sales_quantitycov#83] -(42) TakeOrderedAndProject +(40) TakeOrderedAndProject Input [15]: [i_item_id#25, i_item_desc#26, s_state#23, store_sales_quantitycount#72, store_sales_quantityave#73, store_sales_quantitystdev#74, store_sales_quantitycov#75, as_store_returns_quantitycount#76, as_store_returns_quantityave#77, as_store_returns_quantitystdev#78, store_returns_quantitycov#79, catalog_sales_quantitycount#80, catalog_sales_quantityave#81, catalog_sales_quantitystdev#82, catalog_sales_quantitycov#83] Arguments: 100, [i_item_id#25 ASC NULLS FIRST, i_item_desc#26 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#25, i_item_desc#26, s_state#23, store_sales_quantitycount#72, store_sales_quantityave#73, store_sales_quantitystdev#74, store_sales_quantitycov#75, as_store_returns_quantitycount#76, as_store_returns_quantityave#77, as_store_returns_quantitystdev#78, store_returns_quantitycov#79, catalog_sales_quantitycount#80, catalog_sales_quantityave#81, catalog_sales_quantitystdev#82, catalog_sales_quantitycov#83] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) (unknown) Scan parquet spark_catalog.default.date_dim @@ -256,27 +248,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(42) CometFilter Input [2]: [d_date_sk#19, d_quarter_name#84] Condition : ((isnotnull(d_quarter_name#84) AND (d_quarter_name#84 = 2001Q1)) AND isnotnull(d_date_sk#19)) -(45) CometProject +(43) CometProject Input [2]: [d_date_sk#19, d_quarter_name#84] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(47) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim @@ -286,18 +278,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter +(47) CometFilter Input [2]: [d_date_sk#20, d_quarter_name#85] Condition : (d_quarter_name#85 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#20)) -(50) CometProject +(48) CometProject Input [2]: [d_date_sk#20, d_quarter_name#85] Arguments: [d_date_sk#20], [d_date_sk#20] -(51) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#20] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index 13d7ab8775..9f4d67decc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -1,78 +1,76 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] WholeStageCodegen (9) HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_state] #1 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + InputAdapter + Exchange [i_item_id,i_item_desc,s_state] #1 + WholeStageCodegen (8) + HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] + Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt index c248a76233..613377a61a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt @@ -1,49 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- * ColumnarToRow (43) - +- CometColumnarExchange (42) - +- RowToColumnar (41) - +- * HashAggregate (40) - +- * Expand (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Project (17) - : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- CometProject (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : : +- BroadcastExchange (15) - : : : : +- * ColumnarToRow (14) - : : : : +- CometProject (13) - : : : : +- CometFilter (12) - : : : : +- CometScan parquet spark_catalog.default.customer (11) - : : : +- BroadcastExchange (21) - : : : +- * ColumnarToRow (20) - : : : +- CometFilter (19) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) - : : +- BroadcastExchange (27) - : : +- * ColumnarToRow (26) - : : +- CometFilter (25) - : : +- CometScan parquet spark_catalog.default.customer_address (24) - : +- ReusedExchange (30) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.item (33) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Expand (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * Project (17) + : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : :- * Project (10) + : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : +- BroadcastExchange (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- CometProject (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : : : +- BroadcastExchange (15) + : : : : +- * ColumnarToRow (14) + : : : : +- CometProject (13) + : : : : +- CometFilter (12) + : : : : +- CometScan parquet spark_catalog.default.customer (11) + : : : +- BroadcastExchange (21) + : : : +- * ColumnarToRow (20) + : : : +- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) + : : +- BroadcastExchange (27) + : : +- * ColumnarToRow (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.customer_address (24) + : +- ReusedExchange (30) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.item (33) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -181,7 +179,7 @@ Join condition: None Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -(30) ReusedExchange [Reuses operator id: 50] +(30) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 7] @@ -233,35 +231,29 @@ Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(c Aggregate Attributes [14]: [sum#33, count#34, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46] Results [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] -(41) RowToColumnar +(41) Exchange Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] +Arguments: hashpartitioning(i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(42) CometColumnarExchange -Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] -Arguments: hashpartitioning(i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) ColumnarToRow [codegen id : 8] -Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] - -(44) HashAggregate [codegen id : 8] +(42) HashAggregate [codegen id : 8] Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] Keys [5]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#61, avg(cast(cs_list_price#5 as decimal(12,2)))#62, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63, avg(cast(cs_sales_price#6 as decimal(12,2)))#64, avg(cast(cs_net_profit#8 as decimal(12,2)))#65, avg(cast(c_birth_year#19 as decimal(12,2)))#66, avg(cast(cd_dep_count#14 as decimal(12,2)))#67] Results [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, avg(cast(cs_quantity#4 as decimal(12,2)))#61 AS agg1#68, avg(cast(cs_list_price#5 as decimal(12,2)))#62 AS agg2#69, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63 AS agg3#70, avg(cast(cs_sales_price#6 as decimal(12,2)))#64 AS agg4#71, avg(cast(cs_net_profit#8 as decimal(12,2)))#65 AS agg5#72, avg(cast(c_birth_year#19 as decimal(12,2)))#66 AS agg6#73, avg(cast(cd_dep_count#14 as decimal(12,2)))#67 AS agg7#74] -(45) TakeOrderedAndProject +(43) TakeOrderedAndProject Input [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] Arguments: 100, [ca_country#29 ASC NULLS FIRST, ca_state#30 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) (unknown) Scan parquet spark_catalog.default.date_dim @@ -271,18 +263,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(45) CometFilter Input [2]: [d_date_sk#25, d_year#75] Condition : ((isnotnull(d_year#75) AND (d_year#75 = 1998)) AND isnotnull(d_date_sk#25)) -(48) CometProject +(46) CometProject Input [2]: [d_date_sk#25, d_year#75] Arguments: [d_date_sk#25], [d_date_sk#25] -(49) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(50) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt index 06e0f4d108..47911b9ba3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt @@ -1,73 +1,71 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] WholeStageCodegen (8) HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + InputAdapter + Exchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 + WholeStageCodegen (7) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Expand [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt index 894b5cb6b8..1150b3d665 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt @@ -1,45 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * HashAggregate (40) - +- * ColumnarToRow (39) - +- CometColumnarExchange (38) - +- RowToColumnar (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (17) - : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.date_dim (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan parquet spark_catalog.default.store_sales (5) - : : : +- BroadcastExchange (15) - : : : +- * ColumnarToRow (14) - : : : +- CometProject (13) - : : : +- CometFilter (12) - : : : +- CometScan parquet spark_catalog.default.item (11) - : : +- BroadcastExchange (21) - : : +- * ColumnarToRow (20) - : : +- CometFilter (19) - : : +- CometScan parquet spark_catalog.default.customer (18) - : +- BroadcastExchange (27) - : +- * ColumnarToRow (26) - : +- CometFilter (25) - : +- CometScan parquet spark_catalog.default.customer_address (24) - +- BroadcastExchange (33) - +- * ColumnarToRow (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.store (30) +TakeOrderedAndProject (39) ++- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) + : :- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Project (17) + : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometFilter (6) + : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : +- BroadcastExchange (15) + : : : +- * ColumnarToRow (14) + : : : +- CometProject (13) + : : : +- CometFilter (12) + : : : +- CometScan parquet spark_catalog.default.item (11) + : : +- BroadcastExchange (21) + : : +- * ColumnarToRow (20) + : : +- CometFilter (19) + : : +- CometScan parquet spark_catalog.default.customer (18) + : +- BroadcastExchange (27) + : +- * ColumnarToRow (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer_address (24) + +- BroadcastExchange (33) + +- * ColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.store (30) (unknown) Scan parquet spark_catalog.default.date_dim @@ -212,24 +210,18 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] Aggregate Attributes [1]: [sum#21] Results [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] -(37) RowToColumnar +(37) Exchange Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] +Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(38) CometColumnarExchange -Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] -Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(39) ColumnarToRow [codegen id : 7] -Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] - -(40) HashAggregate [codegen id : 7] +(38) HashAggregate [codegen id : 7] Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] Keys [4]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#23] Results [5]: [i_brand_id#10 AS brand_id#24, i_brand#11 AS brand#25, i_manufact_id#12, i_manufact#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#23,17,2) AS ext_price#26] -(41) TakeOrderedAndProject +(39) TakeOrderedAndProject Input [5]: [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] Arguments: 100, [ext_price#26 DESC NULLS LAST, brand#25 ASC NULLS FIRST, brand_id#24 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt index 3a6991bbb5..c2f5d1a876 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt @@ -1,60 +1,58 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (7) HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] - Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 + WholeStageCodegen (6) + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [s_zip,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] + CometFilter [ca_address_sk,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [s_zip,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt index bbd6ff1aed..2fea53fa79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt @@ -1,45 +1,40 @@ == Physical Plan == -* ColumnarToRow (41) -+- CometSort (40) - +- CometColumnarExchange (39) - +- RowToColumnar (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * HashAggregate (17) - : : +- * ColumnarToRow (16) - : : +- CometColumnarExchange (15) - : : +- RowToColumnar (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * ColumnarToRow (6) - : : : +- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.date_dim (7) - : +- BroadcastExchange (22) - : +- * ColumnarToRow (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan parquet spark_catalog.default.date_dim (18) - +- BroadcastExchange (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * HashAggregate (27) - : +- * ColumnarToRow (26) - : +- ReusedExchange (25) - +- BroadcastExchange (32) - +- * ColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan parquet spark_catalog.default.date_dim (28) +* Sort (36) ++- Exchange (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * ColumnarToRow (6) + : : : +- CometUnion (5) + : : : :- CometProject (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- CometProject (4) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.date_dim (7) + : +- BroadcastExchange (20) + : +- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.date_dim (16) + +- BroadcastExchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * HashAggregate (24) + : +- ReusedExchange (23) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) (unknown) Scan parquet spark_catalog.default.web_sales @@ -106,17 +101,11 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) Aggregate Attributes [7]: [sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18] Results [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] -(14) RowToColumnar +(14) Exchange Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] +Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 8] -Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] - -(17) HashAggregate [codegen id : 8] +(15) HashAggregate [codegen id : 8] Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] @@ -130,38 +119,35 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [2]: [d_week_seq#40, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2001)) AND isnotnull(d_week_seq#40)) -(20) CometProject +(18) CometProject Input [2]: [d_week_seq#40, d_year#41] Arguments: [d_week_seq#40], [d_week_seq#40] -(21) ColumnarToRow [codegen id : 3] +(19) ColumnarToRow [codegen id : 3] Input [1]: [d_week_seq#40] -(22) BroadcastExchange +(20) BroadcastExchange Input [1]: [d_week_seq#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 8] +(21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [d_week_seq#10] Right keys [1]: [d_week_seq#40] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(22) Project [codegen id : 8] Output [8]: [d_week_seq#10 AS d_week_seq1#42, sun_sales#33 AS sun_sales1#43, mon_sales#34 AS mon_sales1#44, tue_sales#35 AS tue_sales1#45, wed_sales#36 AS wed_sales1#46, thu_sales#37 AS thu_sales1#47, fri_sales#38 AS fri_sales1#48, sat_sales#39 AS sat_sales1#49] Input [9]: [d_week_seq#10, sun_sales#33, mon_sales#34, tue_sales#35, wed_sales#36, thu_sales#37, fri_sales#38, sat_sales#39, d_week_seq#40] -(25) ReusedExchange [Reuses operator id: 15] +(23) ReusedExchange [Reuses operator id: 14] Output [8]: [d_week_seq#10, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] -(26) ColumnarToRow [codegen id : 7] -Input [8]: [d_week_seq#10, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] - -(27) HashAggregate [codegen id : 7] +(24) HashAggregate [codegen id : 7] Input [8]: [d_week_seq#10, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] @@ -175,56 +161,50 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct -(29) CometFilter +(26) CometFilter Input [2]: [d_week_seq#57, d_year#58] Condition : ((isnotnull(d_year#58) AND (d_year#58 = 2002)) AND isnotnull(d_week_seq#57)) -(30) CometProject +(27) CometProject Input [2]: [d_week_seq#57, d_year#58] Arguments: [d_week_seq#57], [d_week_seq#57] -(31) ColumnarToRow [codegen id : 6] +(28) ColumnarToRow [codegen id : 6] Input [1]: [d_week_seq#57] -(32) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_week_seq#57] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 7] +(30) BroadcastHashJoin [codegen id : 7] Left keys [1]: [d_week_seq#10] Right keys [1]: [d_week_seq#57] Join type: Inner Join condition: None -(34) Project [codegen id : 7] +(31) Project [codegen id : 7] Output [8]: [d_week_seq#10 AS d_week_seq2#59, sun_sales#33 AS sun_sales2#60, mon_sales#34 AS mon_sales2#61, tue_sales#35 AS tue_sales2#62, wed_sales#36 AS wed_sales2#63, thu_sales#37 AS thu_sales2#64, fri_sales#38 AS fri_sales2#65, sat_sales#39 AS sat_sales2#66] Input [9]: [d_week_seq#10, sun_sales#33, mon_sales#34, tue_sales#35, wed_sales#36, thu_sales#37, fri_sales#38, sat_sales#39, d_week_seq#57] -(35) BroadcastExchange +(32) BroadcastExchange Input [8]: [d_week_seq2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [d_week_seq1#42] Right keys [1]: [(d_week_seq2#59 - 53)] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(34) Project [codegen id : 8] Output [8]: [d_week_seq1#42, round((sun_sales1#43 / sun_sales2#60), 2) AS round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1#44 / mon_sales2#61), 2) AS round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1#45 / tue_sales2#62), 2) AS round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1#46 / wed_sales2#63), 2) AS round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1#47 / thu_sales2#64), 2) AS round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1#48 / fri_sales2#65), 2) AS round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1#49 / sat_sales2#66), 2) AS round((sat_sales1 / sat_sales2), 2)#73] Input [16]: [d_week_seq1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] -(38) RowToColumnar -Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] - -(39) CometColumnarExchange -Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] -Arguments: rangepartitioning(d_week_seq1#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(40) CometSort +(35) Exchange Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] -Arguments: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73], [d_week_seq1#42 ASC NULLS FIRST] +Arguments: rangepartitioning(d_week_seq1#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) ColumnarToRow [codegen id : 9] +(36) Sort [codegen id : 9] Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] +Arguments: [d_week_seq1#42 ASC NULLS FIRST], true, 0 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index d4e1fa96a0..8856ce80d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -1,59 +1,54 @@ WholeStageCodegen (9) - ColumnarToRow + Sort [d_week_seq1] InputAdapter - CometSort [d_week_seq1] - CometColumnarExchange [d_week_seq1] #1 - RowToColumnar - WholeStageCodegen (8) - Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [d_week_seq1,d_week_seq2] - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [d_week_seq] #2 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] - CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] - CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] - CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] + Exchange [d_week_seq1] #1 + WholeStageCodegen (8) + Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] + BroadcastHashJoin [d_week_seq1,d_week_seq2] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + BroadcastHashJoin [d_week_seq,d_week_seq] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + InputAdapter + Exchange [d_week_seq] #2 + WholeStageCodegen (2) + HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] + CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] + CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] + CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + BroadcastHashJoin [d_week_seq,d_week_seq] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + InputAdapter + ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometProject [d_week_seq] + CometFilter [d_year,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index 07df1fbf05..eedf666dd1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -1,28 +1,24 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * ColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- RowToColumnar (18) - +- * HashAggregate (17) - +- * ColumnarToRow (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (20) ++- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -68,7 +64,7 @@ Join condition: None Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 29] +(10) ReusedExchange [Reuses operator id: 25] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -88,57 +84,45 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) RowToColumnar +(14) Exchange Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] - -(17) HashAggregate [codegen id : 4] +(15) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] -(18) RowToColumnar -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] - -(19) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort +(16) Exchange Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6], [i_class#9 ASC NULLS FIRST] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) ColumnarToRow [codegen id : 5] +(17) Sort [codegen id : 5] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(22) Window +(18) Window Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(23) Project [codegen id : 6] +(19) Project [codegen id : 6] Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] -(24) TakeOrderedAndProject +(20) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (25) ++- * ColumnarToRow (24) + +- CometProject (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.date_dim (21) (unknown) Scan parquet spark_catalog.default.date_dim @@ -148,18 +132,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(26) CometFilter +(22) CometFilter Input [2]: [d_date_sk#11, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(27) CometProject +(23) CometProject Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(24) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(29) BroadcastExchange +(25) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index 2b21c3ce4c..52c42bdf2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -4,41 +4,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (5) - ColumnarToRow + Sort [i_class] InputAdapter - CometSort [i_class] - CometColumnarExchange [i_class] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 + Exchange [i_class] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index f559933927..b5625d8e03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -1,30 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (26) -+- * Filter (25) - +- * HashAggregate (24) - +- * ColumnarToRow (23) - +- CometColumnarExchange (22) - +- RowToColumnar (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.warehouse (4) - : +- BroadcastExchange (14) - : +- * ColumnarToRow (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.item (10) - +- ReusedExchange (17) +TakeOrderedAndProject (24) ++- * Filter (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.warehouse (4) + : +- BroadcastExchange (14) + : +- * ColumnarToRow (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.item (10) + +- ReusedExchange (17) (unknown) Scan parquet spark_catalog.default.inventory @@ -102,7 +100,7 @@ Join condition: None Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9] Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#9] -(17) ReusedExchange [Reuses operator id: 30] +(17) ReusedExchange [Reuses operator id: 28] Output [2]: [d_date_sk#11, d_date#12] (18) BroadcastHashJoin [codegen id : 4] @@ -122,38 +120,32 @@ Functions [2]: [partial_sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity Aggregate Attributes [2]: [sum#13, sum#14] Results [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] -(21) RowToColumnar +(21) Exchange Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] +Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) CometColumnarExchange -Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) ColumnarToRow [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] - -(24) HashAggregate [codegen id : 5] +(22) HashAggregate [codegen id : 5] Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] Keys [2]: [w_warehouse_name#7, i_item_id#9] Functions [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] Aggregate Attributes [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18] Results [4]: [w_warehouse_name#7, i_item_id#9, sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17 AS inv_before#19, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_after#20] -(25) Filter [codegen id : 5] +(23) Filter [codegen id : 5] Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] Condition : (CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667) END AND CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5) END) -(26) TakeOrderedAndProject +(24) TakeOrderedAndProject Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (30) -+- * ColumnarToRow (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.date_dim (27) +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) (unknown) Scan parquet spark_catalog.default.date_dim @@ -163,14 +155,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter +(26) CometFilter Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-10)) AND (d_date#12 <= 2000-04-10)) AND isnotnull(d_date_sk#11)) -(29) ColumnarToRow [codegen id : 1] +(27) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_date#12] -(30) BroadcastExchange +(28) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index 016ab7b91b..e20755e12f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -2,43 +2,41 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (5) Filter [inv_before,inv_after] HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,i_item_id] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_warehouse_sk,inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + Exchange [w_warehouse_name,i_item_id] #1 + WholeStageCodegen (4) + HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] + Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_warehouse_sk,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - ReusedExchange [d_date_sk,d_date] #2 + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_item_id] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + InputAdapter + ReusedExchange [d_date_sk,d_date] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt index 0513ff1811..7dfa2dc06a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt @@ -1,29 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- * ColumnarToRow (23) - +- CometColumnarExchange (22) - +- RowToColumnar (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.warehouse (13) +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Expand (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + +- BroadcastExchange (16) + +- * ColumnarToRow (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.warehouse (13) (unknown) Scan parquet spark_catalog.default.inventory @@ -41,7 +39,7 @@ Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (3) ColumnarToRow [codegen id : 4] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(4) ReusedExchange [Reuses operator id: 30] +(4) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 4] @@ -121,35 +119,29 @@ Functions [1]: [partial_avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [sum#18, count#19] Results [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] -(21) RowToColumnar +(21) Exchange Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] +Arguments: hashpartitioning(i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) CometColumnarExchange -Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] -Arguments: hashpartitioning(i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) ColumnarToRow [codegen id : 5] -Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] - -(24) HashAggregate [codegen id : 5] +(22) HashAggregate [codegen id : 5] Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] Keys [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] Functions [1]: [avg(inv_quantity_on_hand#3)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#22] Results [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, avg(inv_quantity_on_hand#3)#22 AS qoh#23] -(25) TakeOrderedAndProject +(23) TakeOrderedAndProject Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] Arguments: 100, [qoh#23 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (30) -+- * ColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.date_dim (26) +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) (unknown) Scan parquet spark_catalog.default.date_dim @@ -159,18 +151,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter +(25) CometFilter Input [2]: [d_date_sk#6, d_month_seq#24] Condition : (((isnotnull(d_month_seq#24) AND (d_month_seq#24 >= 1200)) AND (d_month_seq#24 <= 1211)) AND isnotnull(d_date_sk#6)) -(28) CometProject +(26) CometProject Input [2]: [d_date_sk#6, d_month_seq#24] Arguments: [d_date_sk#6], [d_date_sk#6] -(29) ColumnarToRow [codegen id : 1] +(27) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(30) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt index cfa2bd703b..92714bb02d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt @@ -1,44 +1,42 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 + InputAdapter + Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index 763f886869..5ec1794c1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -1,81 +1,70 @@ == Physical Plan == -* HashAggregate (77) -+- * ColumnarToRow (76) - +- CometColumnarExchange (75) - +- RowToColumnar (74) - +- * HashAggregate (73) - +- Union (72) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (48) - : : +- * SortMergeJoin LeftSemi (47) - : : :- * ColumnarToRow (28) - : : : +- CometSort (27) - : : : +- CometColumnarExchange (26) - : : : +- RowToColumnar (25) - : : : +- * Project (24) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : : :- * ColumnarToRow (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (22) - : : : +- * Project (21) - : : : +- * Filter (20) - : : : +- * HashAggregate (19) - : : : +- * ColumnarToRow (18) - : : : +- CometColumnarExchange (17) - : : : +- RowToColumnar (16) - : : : +- * HashAggregate (15) - : : : +- * Project (14) - : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : :- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (12) - : : : +- * ColumnarToRow (11) - : : : +- CometFilter (10) - : : : +- CometScan parquet spark_catalog.default.item (9) - : : +- * Sort (46) - : : +- * Project (45) - : : +- * Filter (44) - : : +- * HashAggregate (43) - : : +- * ColumnarToRow (42) - : : +- CometColumnarExchange (41) - : : +- RowToColumnar (40) - : : +- * HashAggregate (39) - : : +- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * ColumnarToRow (32) - : : : +- CometProject (31) - : : : +- CometFilter (30) - : : : +- CometScan parquet spark_catalog.default.store_sales (29) - : : +- BroadcastExchange (36) - : : +- * ColumnarToRow (35) - : : +- CometFilter (34) - : : +- CometScan parquet spark_catalog.default.customer (33) - : +- ReusedExchange (49) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * SortMergeJoin LeftSemi (67) - : :- * ColumnarToRow (60) - : : +- CometSort (59) - : : +- CometColumnarExchange (58) - : : +- RowToColumnar (57) - : : +- * Project (56) - : : +- * BroadcastHashJoin LeftSemi BuildRight (55) - : : :- * ColumnarToRow (53) - : : : +- CometScan parquet spark_catalog.default.web_sales (52) - : : +- ReusedExchange (54) - : +- * Sort (66) - : +- * Project (65) - : +- * Filter (64) - : +- * HashAggregate (63) - : +- * ColumnarToRow (62) - : +- ReusedExchange (61) - +- ReusedExchange (69) +* HashAggregate (66) ++- Exchange (65) + +- * HashAggregate (64) + +- Union (63) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (42) + : : +- * SortMergeJoin LeftSemi (41) + : : :- * Sort (24) + : : : +- Exchange (23) + : : : +- * Project (22) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (21) + : : : :- * ColumnarToRow (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- Exchange (16) + : : : +- * HashAggregate (15) + : : : +- * Project (14) + : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : :- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (12) + : : : +- * ColumnarToRow (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.item (9) + : : +- * Sort (40) + : : +- * Project (39) + : : +- * Filter (38) + : : +- * HashAggregate (37) + : : +- Exchange (36) + : : +- * HashAggregate (35) + : : +- * Project (34) + : : +- * BroadcastHashJoin Inner BuildRight (33) + : : :- * ColumnarToRow (28) + : : : +- CometProject (27) + : : : +- CometFilter (26) + : : : +- CometScan parquet spark_catalog.default.store_sales (25) + : : +- BroadcastExchange (32) + : : +- * ColumnarToRow (31) + : : +- CometFilter (30) + : : +- CometScan parquet spark_catalog.default.customer (29) + : +- ReusedExchange (43) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) + :- * Project (59) + : +- * SortMergeJoin LeftSemi (58) + : :- * Sort (52) + : : +- Exchange (51) + : : +- * Project (50) + : : +- * BroadcastHashJoin LeftSemi BuildRight (49) + : : :- * ColumnarToRow (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- ReusedExchange (48) + : +- * Sort (57) + : +- * Project (56) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- ReusedExchange (53) + +- ReusedExchange (60) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -103,7 +92,7 @@ Condition : isnotnull(ss_item_sk#7) (5) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(6) ReusedExchange [Reuses operator id: 87] +(6) ReusedExchange [Reuses operator id: 76] Output [2]: [d_date_sk#10, d_date#11] (7) BroadcastHashJoin [codegen id : 3] @@ -151,58 +140,46 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(16) RowToColumnar +(16) Exchange Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] +Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) CometColumnarExchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(18) ColumnarToRow [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(19) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 4] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(20) Filter [codegen id : 4] +(18) Filter [codegen id : 4] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(21) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] -(22) BroadcastExchange +(20) BroadcastExchange Input [1]: [item_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 5] +(21) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 5] +(22) Project [codegen id : 5] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(25) RowToColumnar -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(26) CometColumnarExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort +(23) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) ColumnarToRow [codegen id : 6] +(24) Sort [codegen id : 6] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] @@ -211,15 +188,15 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(30) CometFilter +(26) CometFilter Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Condition : isnotnull(ss_customer_sk#20) -(31) CometProject +(27) CometProject Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -(32) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 8] Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] (unknown) Scan parquet spark_catalog.default.customer @@ -229,83 +206,77 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(34) CometFilter +(30) CometFilter Input [1]: [c_customer_sk#24] Condition : isnotnull(c_customer_sk#24) -(35) ColumnarToRow [codegen id : 7] +(31) ColumnarToRow [codegen id : 7] Input [1]: [c_customer_sk#24] -(36) BroadcastExchange +(32) BroadcastExchange Input [1]: [c_customer_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 8] +(33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#20] Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(38) Project [codegen id : 8] +(34) Project [codegen id : 8] Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -(39) HashAggregate [codegen id : 8] +(35) HashAggregate [codegen id : 8] Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Keys [1]: [c_customer_sk#24] Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [2]: [sum#25, isEmpty#26] Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(40) RowToColumnar -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] - -(41) CometColumnarExchange +(36) Exchange Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(42) ColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] - -(43) HashAggregate [codegen id : 9] +(37) HashAggregate [codegen id : 9] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(44) Filter [codegen id : 9] +(38) Filter [codegen id : 9] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) -(45) Project [codegen id : 9] +(39) Project [codegen id : 9] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(46) Sort [codegen id : 9] +(40) Sort [codegen id : 9] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(47) SortMergeJoin [codegen id : 11] +(41) SortMergeJoin [codegen id : 11] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(48) Project [codegen id : 11] +(42) Project [codegen id : 11] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(49) ReusedExchange [Reuses operator id: 82] +(43) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#33] -(50) BroadcastHashJoin [codegen id : 11] +(44) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(51) Project [codegen id : 11] +(45) Project [codegen id : 11] Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#34] Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#33] @@ -316,104 +287,89 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#39), dynamicpruningexpression(ws_sold_date_sk#39 IN dynamicpruning#40)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 16] +(47) ColumnarToRow [codegen id : 16] Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(54) ReusedExchange [Reuses operator id: 22] +(48) ReusedExchange [Reuses operator id: 20] Output [1]: [item_sk#18] -(55) BroadcastHashJoin [codegen id : 16] +(49) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_item_sk#35] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(56) Project [codegen id : 16] +(50) Project [codegen id : 16] Output [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(57) RowToColumnar -Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] - -(58) CometColumnarExchange +(51) Exchange Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -Arguments: hashpartitioning(ws_bill_customer_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(ws_bill_customer_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(59) CometSort +(52) Sort [codegen id : 17] Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -Arguments: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39], [ws_bill_customer_sk#36 ASC NULLS FIRST] +Arguments: [ws_bill_customer_sk#36 ASC NULLS FIRST], false, 0 -(60) ColumnarToRow [codegen id : 17] -Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] - -(61) ReusedExchange [Reuses operator id: 41] +(53) ReusedExchange [Reuses operator id: 36] Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(62) ColumnarToRow [codegen id : 20] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] - -(63) HashAggregate [codegen id : 20] +(54) HashAggregate [codegen id : 20] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(64) Filter [codegen id : 20] +(55) Filter [codegen id : 20] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(65) Project [codegen id : 20] +(56) Project [codegen id : 20] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(66) Sort [codegen id : 20] +(57) Sort [codegen id : 20] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(67) SortMergeJoin [codegen id : 22] +(58) SortMergeJoin [codegen id : 22] Left keys [1]: [ws_bill_customer_sk#36] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(68) Project [codegen id : 22] +(59) Project [codegen id : 22] Output [3]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(69) ReusedExchange [Reuses operator id: 82] +(60) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#41] -(70) BroadcastHashJoin [codegen id : 22] +(61) BroadcastHashJoin [codegen id : 22] Left keys [1]: [ws_sold_date_sk#39] Right keys [1]: [d_date_sk#41] Join type: Inner Join condition: None -(71) Project [codegen id : 22] +(62) Project [codegen id : 22] Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#42] Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#41] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 23] +(64) HashAggregate [codegen id : 23] Input [1]: [sales#34] Keys: [] Functions [1]: [partial_sum(sales#34)] Aggregate Attributes [2]: [sum#43, isEmpty#44] Results [2]: [sum#45, isEmpty#46] -(74) RowToColumnar -Input [2]: [sum#45, isEmpty#46] - -(75) CometColumnarExchange -Input [2]: [sum#45, isEmpty#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(76) ColumnarToRow [codegen id : 24] +(65) Exchange Input [2]: [sum#45, isEmpty#46] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(77) HashAggregate [codegen id : 24] +(66) HashAggregate [codegen id : 24] Input [2]: [sum#45, isEmpty#46] Keys: [] Functions [1]: [sum(sales#34)] @@ -423,11 +379,11 @@ Results [1]: [sum(sales#34)#47 AS sum(sales)#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (82) -+- * ColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (71) ++- * ColumnarToRow (70) + +- CometProject (69) + +- CometFilter (68) + +- CometScan parquet spark_catalog.default.date_dim (67) (unknown) Scan parquet spark_catalog.default.date_dim @@ -437,27 +393,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(79) CometFilter +(68) CometFilter Input [3]: [d_date_sk#33, d_year#49, d_moy#50] Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2000)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#33)) -(80) CometProject +(69) CometProject Input [3]: [d_date_sk#33, d_year#49, d_moy#50] Arguments: [d_date_sk#33], [d_date_sk#33] -(81) ColumnarToRow [codegen id : 1] +(70) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#33] -(82) BroadcastExchange +(71) BroadcastExchange Input [1]: [d_date_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (87) -+- * ColumnarToRow (86) - +- CometProject (85) - +- CometFilter (84) - +- CometScan parquet spark_catalog.default.date_dim (83) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim @@ -467,41 +423,37 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(84) CometFilter +(73) CometFilter Input [3]: [d_date_sk#10, d_date#11, d_year#51] Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(85) CometProject +(74) CometProject Input [3]: [d_date_sk#10, d_date#11, d_year#51] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(86) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(87) BroadcastExchange +(76) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (106) -+- * ColumnarToRow (105) - +- CometColumnarExchange (104) - +- RowToColumnar (103) - +- * HashAggregate (102) - +- * HashAggregate (101) - +- * ColumnarToRow (100) - +- CometColumnarExchange (99) - +- RowToColumnar (98) - +- * HashAggregate (97) - +- * Project (96) - +- * BroadcastHashJoin Inner BuildRight (95) - :- * Project (93) - : +- * BroadcastHashJoin Inner BuildRight (92) - : :- * ColumnarToRow (90) - : : +- CometFilter (89) - : : +- CometScan parquet spark_catalog.default.store_sales (88) - : +- ReusedExchange (91) - +- ReusedExchange (94) +Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +* HashAggregate (91) ++- Exchange (90) + +- * HashAggregate (89) + +- * HashAggregate (88) + +- Exchange (87) + +- * HashAggregate (86) + +- * Project (85) + +- * BroadcastHashJoin Inner BuildRight (84) + :- * Project (82) + : +- * BroadcastHashJoin Inner BuildRight (81) + : :- * ColumnarToRow (79) + : : +- CometFilter (78) + : : +- CometScan parquet spark_catalog.default.store_sales (77) + : +- ReusedExchange (80) + +- ReusedExchange (83) (unknown) Scan parquet spark_catalog.default.store_sales @@ -512,93 +464,81 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#55), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(89) CometFilter +(78) CometFilter Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] Condition : isnotnull(ss_customer_sk#52) -(90) ColumnarToRow [codegen id : 3] +(79) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] -(91) ReusedExchange [Reuses operator id: 36] +(80) ReusedExchange [Reuses operator id: 32] Output [1]: [c_customer_sk#57] -(92) BroadcastHashJoin [codegen id : 3] +(81) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#52] Right keys [1]: [c_customer_sk#57] Join type: Inner Join condition: None -(93) Project [codegen id : 3] +(82) Project [codegen id : 3] Output [4]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] Input [5]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] -(94) ReusedExchange [Reuses operator id: 111] +(83) ReusedExchange [Reuses operator id: 96] Output [1]: [d_date_sk#58] -(95) BroadcastHashJoin [codegen id : 3] +(84) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#55] Right keys [1]: [d_date_sk#58] Join type: Inner Join condition: None -(96) Project [codegen id : 3] +(85) Project [codegen id : 3] Output [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] Input [5]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57, d_date_sk#58] -(97) HashAggregate [codegen id : 3] +(86) HashAggregate [codegen id : 3] Input [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] Keys [1]: [c_customer_sk#57] Functions [1]: [partial_sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] Aggregate Attributes [2]: [sum#59, isEmpty#60] Results [3]: [c_customer_sk#57, sum#61, isEmpty#62] -(98) RowToColumnar +(87) Exchange Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] +Arguments: hashpartitioning(c_customer_sk#57, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(99) CometColumnarExchange -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -Arguments: hashpartitioning(c_customer_sk#57, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(100) ColumnarToRow [codegen id : 4] -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] - -(101) HashAggregate [codegen id : 4] +(88) HashAggregate [codegen id : 4] Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] Keys [1]: [c_customer_sk#57] Functions [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] Aggregate Attributes [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63] Results [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63 AS csales#64] -(102) HashAggregate [codegen id : 4] +(89) HashAggregate [codegen id : 4] Input [1]: [csales#64] Keys: [] Functions [1]: [partial_max(csales#64)] Aggregate Attributes [1]: [max#65] Results [1]: [max#66] -(103) RowToColumnar -Input [1]: [max#66] - -(104) CometColumnarExchange -Input [1]: [max#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(105) ColumnarToRow [codegen id : 5] +(90) Exchange Input [1]: [max#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(106) HashAggregate [codegen id : 5] +(91) HashAggregate [codegen id : 5] Input [1]: [max#66] Keys: [] Functions [1]: [max(csales#64)] Aggregate Attributes [1]: [max(csales#64)#67] Results [1]: [max(csales#64)#67 AS tpcds_cmax#68] -Subquery:4 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#55 IN dynamicpruning#56 -BroadcastExchange (111) -+- * ColumnarToRow (110) - +- CometProject (109) - +- CometFilter (108) - +- CometScan parquet spark_catalog.default.date_dim (107) +Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#55 IN dynamicpruning#56 +BroadcastExchange (96) ++- * ColumnarToRow (95) + +- CometProject (94) + +- CometFilter (93) + +- CometScan parquet spark_catalog.default.date_dim (92) (unknown) Scan parquet spark_catalog.default.date_dim @@ -608,23 +548,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(108) CometFilter +(93) CometFilter Input [2]: [d_date_sk#58, d_year#69] Condition : (d_year#69 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#58)) -(109) CometProject +(94) CometProject Input [2]: [d_date_sk#58, d_year#69] Arguments: [d_date_sk#58], [d_date_sk#58] -(110) ColumnarToRow [codegen id : 1] +(95) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#58] -(111) BroadcastExchange +(96) BroadcastExchange Input [1]: [d_date_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index e49c73087f..0ec56d0e72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -1,170 +1,155 @@ WholeStageCodegen (24) HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (23) - HashAggregate [sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (11) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometSort [cs_bill_customer_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - RowToColumnar - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - WholeStageCodegen (9) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - ColumnarToRow - InputAdapter - CometColumnarExchange #10 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [c_customer_sk] #9 - InputAdapter - ReusedExchange [d_date_sk] #12 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + InputAdapter + Exchange #1 + WholeStageCodegen (23) + HashAggregate [sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (11) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (6) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #2 + WholeStageCodegen (5) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometColumnarExchange [c_customer_sk] #8 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + InputAdapter + Exchange [_groupingexpression,i_item_sk,d_date] #5 + WholeStageCodegen (3) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (17) - ColumnarToRow - InputAdapter - CometSort [ws_bill_customer_sk] - CometColumnarExchange [ws_bill_customer_sk] #13 - RowToColumnar - WholeStageCodegen (16) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] + WholeStageCodegen (9) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + Subquery #3 + WholeStageCodegen (5) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + Exchange #10 + WholeStageCodegen (4) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] + InputAdapter + Exchange [c_customer_sk] #11 + WholeStageCodegen (3) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [c_customer_sk] #9 + InputAdapter + ReusedExchange [d_date_sk] #12 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + InputAdapter + Exchange [c_customer_sk] #8 + WholeStageCodegen (8) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] InputAdapter - ReusedExchange [item_sk] #4 - InputAdapter - WholeStageCodegen (20) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + BroadcastExchange #9 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (22) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (17) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #13 + WholeStageCodegen (16) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #3 + WholeStageCodegen (20) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 3c3ac56e1f..8ffb43cf53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -1,105 +1,91 @@ == Physical Plan == -TakeOrderedAndProject (101) -+- Union (100) - :- * HashAggregate (71) - : +- * ColumnarToRow (70) - : +- CometColumnarExchange (69) - : +- RowToColumnar (68) - : +- * HashAggregate (67) - : +- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * SortMergeJoin LeftSemi (48) - : : : :- * ColumnarToRow (29) - : : : : +- CometSort (28) - : : : : +- CometColumnarExchange (27) - : : : : +- RowToColumnar (26) - : : : : +- * Project (25) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (24) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (23) - : : : : +- * Project (22) - : : : : +- * Filter (21) - : : : : +- * HashAggregate (20) - : : : : +- * ColumnarToRow (19) - : : : : +- CometColumnarExchange (18) - : : : : +- RowToColumnar (17) - : : : : +- * HashAggregate (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (7) - : : : : +- BroadcastExchange (13) - : : : : +- * ColumnarToRow (12) - : : : : +- CometFilter (11) - : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : +- * Sort (47) - : : : +- * Project (46) - : : : +- * Filter (45) - : : : +- * HashAggregate (44) - : : : +- * ColumnarToRow (43) - : : : +- CometColumnarExchange (42) - : : : +- RowToColumnar (41) - : : : +- * HashAggregate (40) - : : : +- * Project (39) - : : : +- * BroadcastHashJoin Inner BuildRight (38) - : : : :- * ColumnarToRow (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan parquet spark_catalog.default.store_sales (30) - : : : +- BroadcastExchange (37) - : : : +- * ColumnarToRow (36) - : : : +- CometFilter (35) - : : : +- CometScan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (61) - : : +- * SortMergeJoin LeftSemi (60) - : : :- * ColumnarToRow (53) - : : : +- CometSort (52) - : : : +- CometColumnarExchange (51) - : : : +- CometFilter (50) - : : : +- CometScan parquet spark_catalog.default.customer (49) - : : +- * Sort (59) - : : +- * Project (58) - : : +- * Filter (57) - : : +- * HashAggregate (56) - : : +- * ColumnarToRow (55) - : : +- ReusedExchange (54) - : +- ReusedExchange (64) - +- * HashAggregate (99) - +- * ColumnarToRow (98) - +- CometColumnarExchange (97) - +- RowToColumnar (96) - +- * HashAggregate (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * SortMergeJoin LeftSemi (88) - : : :- * ColumnarToRow (81) - : : : +- CometSort (80) - : : : +- CometColumnarExchange (79) - : : : +- RowToColumnar (78) - : : : +- * Project (77) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (76) - : : : :- * ColumnarToRow (74) - : : : : +- CometFilter (73) - : : : : +- CometScan parquet spark_catalog.default.web_sales (72) - : : : +- ReusedExchange (75) - : : +- * Sort (87) - : : +- * Project (86) - : : +- * Filter (85) - : : +- * HashAggregate (84) - : : +- * ColumnarToRow (83) - : : +- ReusedExchange (82) - : +- ReusedExchange (89) - +- ReusedExchange (92) +TakeOrderedAndProject (87) ++- Union (86) + :- * HashAggregate (62) + : +- Exchange (61) + : +- * HashAggregate (60) + : +- * Project (59) + : +- * BroadcastHashJoin Inner BuildRight (58) + : :- * Project (56) + : : +- * BroadcastHashJoin Inner BuildRight (55) + : : :- * SortMergeJoin LeftSemi (42) + : : : :- * Sort (25) + : : : : +- Exchange (24) + : : : : +- * Project (23) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (21) + : : : : +- * Project (20) + : : : : +- * Filter (19) + : : : : +- * HashAggregate (18) + : : : : +- Exchange (17) + : : : : +- * HashAggregate (16) + : : : : +- * Project (15) + : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : :- * Project (9) + : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : :- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : : +- ReusedExchange (7) + : : : : +- BroadcastExchange (13) + : : : : +- * ColumnarToRow (12) + : : : : +- CometFilter (11) + : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : +- * Sort (41) + : : : +- * Project (40) + : : : +- * Filter (39) + : : : +- * HashAggregate (38) + : : : +- Exchange (37) + : : : +- * HashAggregate (36) + : : : +- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (29) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan parquet spark_catalog.default.store_sales (26) + : : : +- BroadcastExchange (33) + : : : +- * ColumnarToRow (32) + : : : +- CometFilter (31) + : : : +- CometScan parquet spark_catalog.default.customer (30) + : : +- BroadcastExchange (54) + : : +- * SortMergeJoin LeftSemi (53) + : : :- * Sort (47) + : : : +- Exchange (46) + : : : +- * ColumnarToRow (45) + : : : +- CometFilter (44) + : : : +- CometScan parquet spark_catalog.default.customer (43) + : : +- * Sort (52) + : : +- * Project (51) + : : +- * Filter (50) + : : +- * HashAggregate (49) + : : +- ReusedExchange (48) + : +- ReusedExchange (57) + +- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Project (82) + +- * BroadcastHashJoin Inner BuildRight (81) + :- * Project (79) + : +- * BroadcastHashJoin Inner BuildRight (78) + : :- * SortMergeJoin LeftSemi (76) + : : :- * Sort (70) + : : : +- Exchange (69) + : : : +- * Project (68) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (67) + : : : :- * ColumnarToRow (65) + : : : : +- CometFilter (64) + : : : : +- CometScan parquet spark_catalog.default.web_sales (63) + : : : +- ReusedExchange (66) + : : +- * Sort (75) + : : +- * Project (74) + : : +- * Filter (73) + : : +- * HashAggregate (72) + : : +- ReusedExchange (71) + : +- ReusedExchange (77) + +- ReusedExchange (80) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -132,7 +118,7 @@ Condition : isnotnull(ss_item_sk#7) (6) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(7) ReusedExchange [Reuses operator id: 111] +(7) ReusedExchange [Reuses operator id: 97] Output [2]: [d_date_sk#10, d_date#11] (8) BroadcastHashJoin [codegen id : 3] @@ -180,58 +166,46 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(17) RowToColumnar +(17) Exchange Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] +Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(18) CometColumnarExchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(19) ColumnarToRow [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(20) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 4] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(21) Filter [codegen id : 4] +(19) Filter [codegen id : 4] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(22) Project [codegen id : 4] +(20) Project [codegen id : 4] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] -(23) BroadcastExchange +(21) BroadcastExchange Input [1]: [item_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 5] +(22) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(25) Project [codegen id : 5] +(23) Project [codegen id : 5] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(26) RowToColumnar -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(27) CometColumnarExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(28) CometSort +(24) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) ColumnarToRow [codegen id : 6] +(25) Sort [codegen id : 6] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] @@ -240,15 +214,15 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(31) CometFilter +(27) CometFilter Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Condition : isnotnull(ss_customer_sk#20) -(32) CometProject +(28) CometProject Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -(33) ColumnarToRow [codegen id : 8] +(29) ColumnarToRow [codegen id : 8] Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] (unknown) Scan parquet spark_catalog.default.customer @@ -258,64 +232,58 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [1]: [c_customer_sk#24] Condition : isnotnull(c_customer_sk#24) -(36) ColumnarToRow [codegen id : 7] +(32) ColumnarToRow [codegen id : 7] Input [1]: [c_customer_sk#24] -(37) BroadcastExchange +(33) BroadcastExchange Input [1]: [c_customer_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#20] Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(39) Project [codegen id : 8] +(35) Project [codegen id : 8] Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -(40) HashAggregate [codegen id : 8] +(36) HashAggregate [codegen id : 8] Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Keys [1]: [c_customer_sk#24] Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [2]: [sum#25, isEmpty#26] Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(41) RowToColumnar +(37) Exchange Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(42) CometColumnarExchange -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) ColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] - -(44) HashAggregate [codegen id : 9] +(38) HashAggregate [codegen id : 9] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(45) Filter [codegen id : 9] +(39) Filter [codegen id : 9] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) -(46) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(47) Sort [codegen id : 9] +(41) Sort [codegen id : 9] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(48) SortMergeJoin [codegen id : 16] +(42) SortMergeJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi @@ -328,97 +296,88 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(50) CometFilter +(44) CometFilter Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] Condition : isnotnull(c_customer_sk#33) -(51) CometColumnarExchange +(45) ColumnarToRow [codegen id : 10] Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: hashpartitioning(c_customer_sk#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(52) CometSort +(46) Exchange Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#33, c_first_name#34, c_last_name#35], [c_customer_sk#33 ASC NULLS FIRST] +Arguments: hashpartitioning(c_customer_sk#33, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(53) ColumnarToRow [codegen id : 10] +(47) Sort [codegen id : 11] Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +Arguments: [c_customer_sk#33 ASC NULLS FIRST], false, 0 -(54) ReusedExchange [Reuses operator id: 42] +(48) ReusedExchange [Reuses operator id: 37] Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(55) ColumnarToRow [codegen id : 13] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] - -(56) HashAggregate [codegen id : 13] +(49) HashAggregate [codegen id : 14] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(57) Filter [codegen id : 13] +(50) Filter [codegen id : 14] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(58) Project [codegen id : 13] +(51) Project [codegen id : 14] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(59) Sort [codegen id : 13] +(52) Sort [codegen id : 14] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(60) SortMergeJoin [codegen id : 14] +(53) SortMergeJoin [codegen id : 15] Left keys [1]: [c_customer_sk#33] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(61) BroadcastExchange +(54) BroadcastExchange Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(62) BroadcastHashJoin [codegen id : 16] +(55) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#33] Join type: Inner Join condition: None -(63) Project [codegen id : 16] +(56) Project [codegen id : 17] Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35] Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#33, c_first_name#34, c_last_name#35] -(64) ReusedExchange [Reuses operator id: 106] +(57) ReusedExchange [Reuses operator id: 92] Output [1]: [d_date_sk#36] -(65) BroadcastHashJoin [codegen id : 16] +(58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(66) Project [codegen id : 16] +(59) Project [codegen id : 17] Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35, d_date_sk#36] -(67) HashAggregate [codegen id : 16] +(60) HashAggregate [codegen id : 17] Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] Aggregate Attributes [2]: [sum#37, isEmpty#38] Results [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -(68) RowToColumnar -Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] - -(69) CometColumnarExchange -Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(70) ColumnarToRow [codegen id : 17] +(61) Exchange Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] +Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(71) HashAggregate [codegen id : 17] +(62) HashAggregate [codegen id : 18] Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] @@ -433,135 +392,120 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(73) CometFilter +(64) CometFilter Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Condition : isnotnull(ws_bill_customer_sk#44) -(74) ColumnarToRow [codegen id : 22] +(65) ColumnarToRow [codegen id : 23] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(75) ReusedExchange [Reuses operator id: 23] +(66) ReusedExchange [Reuses operator id: 21] Output [1]: [item_sk#18] -(76) BroadcastHashJoin [codegen id : 22] +(67) BroadcastHashJoin [codegen id : 23] Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(77) Project [codegen id : 22] +(68) Project [codegen id : 23] Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(78) RowToColumnar -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] - -(79) CometColumnarExchange -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(80) CometSort +(69) Exchange Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47], [ws_bill_customer_sk#44 ASC NULLS FIRST] +Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(81) ColumnarToRow [codegen id : 23] +(70) Sort [codegen id : 24] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 -(82) ReusedExchange [Reuses operator id: 42] +(71) ReusedExchange [Reuses operator id: 37] Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(83) ColumnarToRow [codegen id : 26] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] - -(84) HashAggregate [codegen id : 26] +(72) HashAggregate [codegen id : 27] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(85) Filter [codegen id : 26] +(73) Filter [codegen id : 27] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(86) Project [codegen id : 26] +(74) Project [codegen id : 27] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(87) Sort [codegen id : 26] +(75) Sort [codegen id : 27] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(88) SortMergeJoin [codegen id : 33] +(76) SortMergeJoin [codegen id : 35] Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(89) ReusedExchange [Reuses operator id: 61] +(77) ReusedExchange [Reuses operator id: 54] Output [3]: [c_customer_sk#49, c_first_name#50, c_last_name#51] -(90) BroadcastHashJoin [codegen id : 33] +(78) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#49] Join type: Inner Join condition: None -(91) Project [codegen id : 33] +(79) Project [codegen id : 35] Output [5]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#50, c_last_name#51] Input [7]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_customer_sk#49, c_first_name#50, c_last_name#51] -(92) ReusedExchange [Reuses operator id: 106] +(80) ReusedExchange [Reuses operator id: 92] Output [1]: [d_date_sk#52] -(93) BroadcastHashJoin [codegen id : 33] +(81) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#52] Join type: Inner Join condition: None -(94) Project [codegen id : 33] +(82) Project [codegen id : 35] Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] Input [6]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#50, c_last_name#51, d_date_sk#52] -(95) HashAggregate [codegen id : 33] +(83) HashAggregate [codegen id : 35] Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] Keys [2]: [c_last_name#51, c_first_name#50] Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] Aggregate Attributes [2]: [sum#53, isEmpty#54] Results [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] -(96) RowToColumnar +(84) Exchange Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] +Arguments: hashpartitioning(c_last_name#51, c_first_name#50, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(97) CometColumnarExchange -Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] -Arguments: hashpartitioning(c_last_name#51, c_first_name#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(98) ColumnarToRow [codegen id : 34] -Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] - -(99) HashAggregate [codegen id : 34] +(85) HashAggregate [codegen id : 36] Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] Keys [2]: [c_last_name#51, c_first_name#50] Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#57] Results [3]: [c_last_name#51, c_first_name#50, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#57 AS sales#58] -(100) Union +(86) Union -(101) TakeOrderedAndProject +(87) TakeOrderedAndProject Input [3]: [c_last_name#35, c_first_name#34, sales#42] Arguments: 100, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, sales#42 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, sales#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (106) -+- * ColumnarToRow (105) - +- CometProject (104) - +- CometFilter (103) - +- CometScan parquet spark_catalog.default.date_dim (102) +BroadcastExchange (92) ++- * ColumnarToRow (91) + +- CometProject (90) + +- CometFilter (89) + +- CometScan parquet spark_catalog.default.date_dim (88) (unknown) Scan parquet spark_catalog.default.date_dim @@ -571,27 +515,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(103) CometFilter +(89) CometFilter Input [3]: [d_date_sk#36, d_year#59, d_moy#60] Condition : ((((isnotnull(d_year#59) AND isnotnull(d_moy#60)) AND (d_year#59 = 2000)) AND (d_moy#60 = 2)) AND isnotnull(d_date_sk#36)) -(104) CometProject +(90) CometProject Input [3]: [d_date_sk#36, d_year#59, d_moy#60] Arguments: [d_date_sk#36], [d_date_sk#36] -(105) ColumnarToRow [codegen id : 1] +(91) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#36] -(106) BroadcastExchange +(92) BroadcastExchange Input [1]: [d_date_sk#36] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (111) -+- * ColumnarToRow (110) - +- CometProject (109) - +- CometFilter (108) - +- CometScan parquet spark_catalog.default.date_dim (107) +BroadcastExchange (97) ++- * ColumnarToRow (96) + +- CometProject (95) + +- CometFilter (94) + +- CometScan parquet spark_catalog.default.date_dim (93) (unknown) Scan parquet spark_catalog.default.date_dim @@ -601,41 +545,37 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(108) CometFilter +(94) CometFilter Input [3]: [d_date_sk#10, d_date#11, d_year#61] Condition : (d_year#61 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(109) CometProject +(95) CometProject Input [3]: [d_date_sk#10, d_date#11, d_year#61] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(110) ColumnarToRow [codegen id : 1] +(96) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(111) BroadcastExchange +(97) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (130) -+- * ColumnarToRow (129) - +- CometColumnarExchange (128) - +- RowToColumnar (127) - +- * HashAggregate (126) - +- * HashAggregate (125) - +- * ColumnarToRow (124) - +- CometColumnarExchange (123) - +- RowToColumnar (122) - +- * HashAggregate (121) - +- * Project (120) - +- * BroadcastHashJoin Inner BuildRight (119) - :- * Project (117) - : +- * BroadcastHashJoin Inner BuildRight (116) - : :- * ColumnarToRow (114) - : : +- CometFilter (113) - : : +- CometScan parquet spark_catalog.default.store_sales (112) - : +- ReusedExchange (115) - +- ReusedExchange (118) +Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +* HashAggregate (112) ++- Exchange (111) + +- * HashAggregate (110) + +- * HashAggregate (109) + +- Exchange (108) + +- * HashAggregate (107) + +- * Project (106) + +- * BroadcastHashJoin Inner BuildRight (105) + :- * Project (103) + : +- * BroadcastHashJoin Inner BuildRight (102) + : :- * ColumnarToRow (100) + : : +- CometFilter (99) + : : +- CometScan parquet spark_catalog.default.store_sales (98) + : +- ReusedExchange (101) + +- ReusedExchange (104) (unknown) Scan parquet spark_catalog.default.store_sales @@ -646,93 +586,81 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#65), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(113) CometFilter +(99) CometFilter Input [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] Condition : isnotnull(ss_customer_sk#62) -(114) ColumnarToRow [codegen id : 3] +(100) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] -(115) ReusedExchange [Reuses operator id: 37] +(101) ReusedExchange [Reuses operator id: 33] Output [1]: [c_customer_sk#67] -(116) BroadcastHashJoin [codegen id : 3] +(102) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#62] Right keys [1]: [c_customer_sk#67] Join type: Inner Join condition: None -(117) Project [codegen id : 3] +(103) Project [codegen id : 3] Output [4]: [ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67] Input [5]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67] -(118) ReusedExchange [Reuses operator id: 135] +(104) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#68] -(119) BroadcastHashJoin [codegen id : 3] +(105) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#65] Right keys [1]: [d_date_sk#68] Join type: Inner Join condition: None -(120) Project [codegen id : 3] +(106) Project [codegen id : 3] Output [3]: [ss_quantity#63, ss_sales_price#64, c_customer_sk#67] Input [5]: [ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67, d_date_sk#68] -(121) HashAggregate [codegen id : 3] +(107) HashAggregate [codegen id : 3] Input [3]: [ss_quantity#63, ss_sales_price#64, c_customer_sk#67] Keys [1]: [c_customer_sk#67] Functions [1]: [partial_sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))] Aggregate Attributes [2]: [sum#69, isEmpty#70] Results [3]: [c_customer_sk#67, sum#71, isEmpty#72] -(122) RowToColumnar -Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] - -(123) CometColumnarExchange -Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] -Arguments: hashpartitioning(c_customer_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(124) ColumnarToRow [codegen id : 4] +(108) Exchange Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] +Arguments: hashpartitioning(c_customer_sk#67, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(125) HashAggregate [codegen id : 4] +(109) HashAggregate [codegen id : 4] Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] Keys [1]: [c_customer_sk#67] Functions [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))] Aggregate Attributes [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))#73] Results [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))#73 AS csales#74] -(126) HashAggregate [codegen id : 4] +(110) HashAggregate [codegen id : 4] Input [1]: [csales#74] Keys: [] Functions [1]: [partial_max(csales#74)] Aggregate Attributes [1]: [max#75] Results [1]: [max#76] -(127) RowToColumnar -Input [1]: [max#76] - -(128) CometColumnarExchange -Input [1]: [max#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(129) ColumnarToRow [codegen id : 5] +(111) Exchange Input [1]: [max#76] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] -(130) HashAggregate [codegen id : 5] +(112) HashAggregate [codegen id : 5] Input [1]: [max#76] Keys: [] Functions [1]: [max(csales#74)] Aggregate Attributes [1]: [max(csales#74)#77] Results [1]: [max(csales#74)#77 AS tpcds_cmax#78] -Subquery:4 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#65 IN dynamicpruning#66 -BroadcastExchange (135) -+- * ColumnarToRow (134) - +- CometProject (133) - +- CometFilter (132) - +- CometScan parquet spark_catalog.default.date_dim (131) +Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#65 IN dynamicpruning#66 +BroadcastExchange (117) ++- * ColumnarToRow (116) + +- CometProject (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) (unknown) Scan parquet spark_catalog.default.date_dim @@ -742,25 +670,25 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(132) CometFilter +(114) CometFilter Input [2]: [d_date_sk#68, d_year#79] Condition : (d_year#79 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#68)) -(133) CometProject +(115) CometProject Input [2]: [d_date_sk#68, d_year#79] Arguments: [d_date_sk#68], [d_date_sk#68] -(134) ColumnarToRow [codegen id : 1] +(116) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#68] -(135) BroadcastExchange +(117) BroadcastExchange Input [1]: [d_date_sk#68] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:5 Hosting operator id = 57 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:5 Hosting operator id = 50 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] -Subquery:6 Hosting operator id = 72 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 83cea75a65..49ddeaef82 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -1,204 +1,188 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name] #1 - RowToColumnar - WholeStageCodegen (16) - HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] - Project [cs_quantity,cs_list_price,c_first_name,c_last_name] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometSort [cs_bill_customer_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - RowToColumnar - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - WholeStageCodegen (9) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - ColumnarToRow - InputAdapter - CometColumnarExchange #10 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [c_customer_sk] #9 - InputAdapter - ReusedExchange [d_date_sk] #12 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name] #1 + WholeStageCodegen (17) + HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] + Project [cs_quantity,cs_list_price,c_first_name,c_last_name] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (6) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #2 + WholeStageCodegen (5) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometColumnarExchange [c_customer_sk] #8 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + InputAdapter + Exchange [_groupingexpression,i_item_sk,d_date] #5 + WholeStageCodegen (3) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (14) - SortMergeJoin [c_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (10) - ColumnarToRow + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + InputAdapter + WholeStageCodegen (9) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + Subquery #3 + WholeStageCodegen (5) + HashAggregate [max] [max(csales),tpcds_cmax,max] InputAdapter - CometSort [c_customer_sk] - CometColumnarExchange [c_customer_sk] #14 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - WholeStageCodegen (13) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] - ColumnarToRow - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (34) - HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name] #15 - RowToColumnar - WholeStageCodegen (33) - HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] - Project [ws_quantity,ws_list_price,c_first_name,c_last_name] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometSort [ws_bill_customer_sk] - CometColumnarExchange [ws_bill_customer_sk] #16 - RowToColumnar - WholeStageCodegen (22) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] + Exchange #10 + WholeStageCodegen (4) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] + InputAdapter + Exchange [c_customer_sk] #11 + WholeStageCodegen (3) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [c_customer_sk] #9 + InputAdapter + ReusedExchange [d_date_sk] #12 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + InputAdapter + Exchange [c_customer_sk] #8 + WholeStageCodegen (8) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] InputAdapter - ReusedExchange [item_sk] #4 - InputAdapter - WholeStageCodegen (26) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] - ColumnarToRow + BroadcastExchange #9 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (15) + SortMergeJoin [c_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (11) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #14 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + WholeStageCodegen (14) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter ReusedExchange [c_customer_sk,sum,isEmpty] #8 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (36) + HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name] #15 + WholeStageCodegen (35) + HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] + Project [ws_quantity,ws_list_price,c_first_name,c_last_name] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (24) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #16 + WholeStageCodegen (23) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + InputAdapter + WholeStageCodegen (27) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index 51943ef8bb..d2534e9be0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -1,53 +1,50 @@ == Physical Plan == -* Filter (49) -+- * HashAggregate (48) - +- * ColumnarToRow (47) - +- CometColumnarExchange (46) - +- RowToColumnar (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- * ColumnarToRow (42) - +- CometColumnarExchange (41) - +- RowToColumnar (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometColumnarExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometColumnarExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.item (21) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_address (33) +* Filter (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store_returns (7) + : : : +- BroadcastExchange (19) + : : : +- * ColumnarToRow (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.store (15) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometFilter (23) + : : +- CometScan parquet spark_catalog.default.item (22) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer (28) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer_address (34) (unknown) Scan parquet spark_catalog.default.store_sales @@ -65,13 +62,16 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) CometColumnarExchange +(4) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) CometSort +(5) Exchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(6) Sort [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] @@ -80,33 +80,34 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(7) CometFilter +(8) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(8) CometProject +(9) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(9) CometColumnarExchange +(10) ColumnarToRow [codegen id : 3] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) CometSort +(11) Exchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(12) Sort [codegen id : 4] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(13) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(13) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(14) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] (unknown) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] @@ -115,28 +116,28 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(15) CometFilter +(16) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(16) CometProject +(17) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(17) ColumnarToRow [codegen id : 1] +(18) ColumnarToRow [codegen id : 5] Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) BroadcastExchange +(19) BroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(21) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -147,24 +148,24 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(22) CometFilter +(23) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) -(23) ColumnarToRow [codegen id : 2] +(24) ColumnarToRow [codegen id : 6] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(24) BroadcastExchange +(25) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(27) Project [codegen id : 9] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] @@ -175,24 +176,24 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(28) CometFilter +(29) CometFilter Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) -(29) ColumnarToRow [codegen id : 3] +(30) ColumnarToRow [codegen id : 7] Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(30) BroadcastExchange +(31) BroadcastExchange Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(31) BroadcastHashJoin [codegen id : 5] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(32) Project [codegen id : 5] +(33) Project [codegen id : 9] Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] @@ -203,152 +204,133 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(34) CometFilter +(35) CometFilter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(35) ColumnarToRow [codegen id : 4] +(36) ColumnarToRow [codegen id : 8] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -(36) BroadcastExchange +(37) BroadcastExchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 5] +(38) BroadcastHashJoin [codegen id : 9] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(38) Project [codegen id : 5] +(39) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(39) HashAggregate [codegen id : 5] +(40) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(40) RowToColumnar +(41) Exchange Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(41) CometColumnarExchange -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(42) ColumnarToRow [codegen id : 6] -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] - -(43) HashAggregate [codegen id : 6] +(42) HashAggregate [codegen id : 10] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(44) HashAggregate [codegen id : 6] +(43) HashAggregate [codegen id : 10] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [partial_sum(netpaid#31)] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(45) RowToColumnar -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] - -(46) CometColumnarExchange +(44) Exchange Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(47) ColumnarToRow [codegen id : 7] -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] - -(48) HashAggregate [codegen id : 7] +(45) HashAggregate [codegen id : 11] Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [sum(netpaid#31)] Aggregate Attributes [1]: [sum(netpaid#31)#36] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(49) Filter [codegen id : 7] +(46) Filter [codegen id : 11] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (81) -+- * ColumnarToRow (80) - +- CometColumnarExchange (79) - +- RowToColumnar (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * ColumnarToRow (75) - +- CometColumnarExchange (74) - +- RowToColumnar (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * ColumnarToRow (56) - : : : : +- CometProject (55) - : : : : +- CometSortMergeJoin (54) - : : : : :- CometSort (51) - : : : : : +- ReusedExchange (50) - : : : : +- CometSort (53) - : : : : +- ReusedExchange (52) - : : : +- ReusedExchange (57) - : : +- BroadcastExchange (63) - : : +- * ColumnarToRow (62) - : : +- CometFilter (61) - : : +- CometScan parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (69) - - -(50) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (73) ++- Exchange (72) + +- * HashAggregate (71) + +- * HashAggregate (70) + +- Exchange (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * SortMergeJoin Inner (51) + : : : : :- * Sort (48) + : : : : : +- ReusedExchange (47) + : : : : +- * Sort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (59) + : : +- * ColumnarToRow (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.item (56) + : +- ReusedExchange (62) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 5] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(51) CometSort +(48) Sort [codegen id : 2] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 -(52) ReusedExchange [Reuses operator id: 9] +(49) ReusedExchange [Reuses operator id: 11] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(53) CometSort +(50) Sort [codegen id : 4] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(54) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(51) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(55) CometProject +(52) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(56) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(57) ReusedExchange [Reuses operator id: 18] +(53) ReusedExchange [Reuses operator id: 19] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(58) BroadcastHashJoin [codegen id : 5] +(54) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(59) Project [codegen id : 5] +(55) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -359,95 +341,83 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(61) CometFilter +(57) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : isnotnull(i_item_sk#15) -(62) ColumnarToRow [codegen id : 2] +(58) ColumnarToRow [codegen id : 6] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(63) BroadcastExchange +(59) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(64) BroadcastHashJoin [codegen id : 5] +(60) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(65) Project [codegen id : 5] +(61) Project [codegen id : 9] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(66) ReusedExchange [Reuses operator id: 30] +(62) ReusedExchange [Reuses operator id: 31] Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(67) BroadcastHashJoin [codegen id : 5] +(63) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(68) Project [codegen id : 5] +(64) Project [codegen id : 9] Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(69) ReusedExchange [Reuses operator id: 36] +(65) ReusedExchange [Reuses operator id: 37] Output [3]: [ca_state#25, ca_zip#26, ca_country#27] -(70) BroadcastHashJoin [codegen id : 5] +(66) BroadcastHashJoin [codegen id : 9] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(71) Project [codegen id : 5] +(67) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(72) HashAggregate [codegen id : 5] +(68) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#40] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] -(73) RowToColumnar -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] - -(74) CometColumnarExchange +(69) Exchange Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(75) ColumnarToRow [codegen id : 6] -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] - -(76) HashAggregate [codegen id : 6] +(70) HashAggregate [codegen id : 10] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(77) HashAggregate [codegen id : 6] +(71) HashAggregate [codegen id : 10] Input [1]: [netpaid#31] Keys: [] Functions [1]: [partial_avg(netpaid#31)] Aggregate Attributes [2]: [sum#42, count#43] Results [2]: [sum#44, count#45] -(78) RowToColumnar -Input [2]: [sum#44, count#45] - -(79) CometColumnarExchange -Input [2]: [sum#44, count#45] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(80) ColumnarToRow [codegen id : 7] +(72) Exchange Input [2]: [sum#44, count#45] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] -(81) HashAggregate [codegen id : 7] +(73) HashAggregate [codegen id : 11] Input [2]: [sum#44, count#45] Keys: [] Functions [1]: [avg(netpaid#31)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index 336a655002..8ebd45fd18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -1,112 +1,118 @@ -WholeStageCodegen (7) +WholeStageCodegen (11) Filter [paid] Subquery #1 - WholeStageCodegen (7) + WholeStageCodegen (11) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #9 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow + InputAdapter + Exchange #9 + WholeStageCodegen (10) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number] #4 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 + BroadcastExchange #11 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #3 + Exchange [ss_ticket_number,ss_item_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_color,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometFilter [i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ca_country,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt index 0425915e84..f9179096ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt @@ -1,53 +1,50 @@ == Physical Plan == -* Filter (49) -+- * HashAggregate (48) - +- * ColumnarToRow (47) - +- CometColumnarExchange (46) - +- RowToColumnar (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- * ColumnarToRow (42) - +- CometColumnarExchange (41) - +- RowToColumnar (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometColumnarExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometColumnarExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.item (21) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_address (33) +* Filter (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store_returns (7) + : : : +- BroadcastExchange (19) + : : : +- * ColumnarToRow (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.store (15) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometFilter (23) + : : +- CometScan parquet spark_catalog.default.item (22) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer (28) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer_address (34) (unknown) Scan parquet spark_catalog.default.store_sales @@ -65,13 +62,16 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) CometColumnarExchange +(4) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) CometSort +(5) Exchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(6) Sort [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] @@ -80,33 +80,34 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(7) CometFilter +(8) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(8) CometProject +(9) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(9) CometColumnarExchange +(10) ColumnarToRow [codegen id : 3] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) CometSort +(11) Exchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(12) Sort [codegen id : 4] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(13) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(13) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(14) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] (unknown) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] @@ -115,28 +116,28 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(15) CometFilter +(16) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(16) CometProject +(17) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(17) ColumnarToRow [codegen id : 1] +(18) ColumnarToRow [codegen id : 5] Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) BroadcastExchange +(19) BroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(21) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -147,24 +148,24 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct -(22) CometFilter +(23) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = chiffon )) AND isnotnull(i_item_sk#15)) -(23) ColumnarToRow [codegen id : 2] +(24) ColumnarToRow [codegen id : 6] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(24) BroadcastExchange +(25) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(27) Project [codegen id : 9] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] @@ -175,24 +176,24 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(28) CometFilter +(29) CometFilter Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) -(29) ColumnarToRow [codegen id : 3] +(30) ColumnarToRow [codegen id : 7] Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(30) BroadcastExchange +(31) BroadcastExchange Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(31) BroadcastHashJoin [codegen id : 5] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(32) Project [codegen id : 5] +(33) Project [codegen id : 9] Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] @@ -203,152 +204,133 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(34) CometFilter +(35) CometFilter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(35) ColumnarToRow [codegen id : 4] +(36) ColumnarToRow [codegen id : 8] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -(36) BroadcastExchange +(37) BroadcastExchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 5] +(38) BroadcastHashJoin [codegen id : 9] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(38) Project [codegen id : 5] +(39) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(39) HashAggregate [codegen id : 5] +(40) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(40) RowToColumnar +(41) Exchange Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(41) CometColumnarExchange -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(42) ColumnarToRow [codegen id : 6] -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] - -(43) HashAggregate [codegen id : 6] +(42) HashAggregate [codegen id : 10] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(44) HashAggregate [codegen id : 6] +(43) HashAggregate [codegen id : 10] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [partial_sum(netpaid#31)] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(45) RowToColumnar -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] - -(46) CometColumnarExchange +(44) Exchange Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(47) ColumnarToRow [codegen id : 7] -Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] - -(48) HashAggregate [codegen id : 7] +(45) HashAggregate [codegen id : 11] Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [sum(netpaid#31)] Aggregate Attributes [1]: [sum(netpaid#31)#36] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(49) Filter [codegen id : 7] +(46) Filter [codegen id : 11] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (81) -+- * ColumnarToRow (80) - +- CometColumnarExchange (79) - +- RowToColumnar (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * ColumnarToRow (75) - +- CometColumnarExchange (74) - +- RowToColumnar (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * ColumnarToRow (56) - : : : : +- CometProject (55) - : : : : +- CometSortMergeJoin (54) - : : : : :- CometSort (51) - : : : : : +- ReusedExchange (50) - : : : : +- CometSort (53) - : : : : +- ReusedExchange (52) - : : : +- ReusedExchange (57) - : : +- BroadcastExchange (63) - : : +- * ColumnarToRow (62) - : : +- CometFilter (61) - : : +- CometScan parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (69) - - -(50) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (73) ++- Exchange (72) + +- * HashAggregate (71) + +- * HashAggregate (70) + +- Exchange (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * SortMergeJoin Inner (51) + : : : : :- * Sort (48) + : : : : : +- ReusedExchange (47) + : : : : +- * Sort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (59) + : : +- * ColumnarToRow (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.item (56) + : +- ReusedExchange (62) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 5] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(51) CometSort +(48) Sort [codegen id : 2] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 -(52) ReusedExchange [Reuses operator id: 9] +(49) ReusedExchange [Reuses operator id: 11] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(53) CometSort +(50) Sort [codegen id : 4] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(54) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(51) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(55) CometProject +(52) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(56) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(57) ReusedExchange [Reuses operator id: 18] +(53) ReusedExchange [Reuses operator id: 19] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(58) BroadcastHashJoin [codegen id : 5] +(54) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(59) Project [codegen id : 5] +(55) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -359,95 +341,83 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(61) CometFilter +(57) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : isnotnull(i_item_sk#15) -(62) ColumnarToRow [codegen id : 2] +(58) ColumnarToRow [codegen id : 6] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(63) BroadcastExchange +(59) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(64) BroadcastHashJoin [codegen id : 5] +(60) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(65) Project [codegen id : 5] +(61) Project [codegen id : 9] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(66) ReusedExchange [Reuses operator id: 30] +(62) ReusedExchange [Reuses operator id: 31] Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(67) BroadcastHashJoin [codegen id : 5] +(63) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(68) Project [codegen id : 5] +(64) Project [codegen id : 9] Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(69) ReusedExchange [Reuses operator id: 36] +(65) ReusedExchange [Reuses operator id: 37] Output [3]: [ca_state#25, ca_zip#26, ca_country#27] -(70) BroadcastHashJoin [codegen id : 5] +(66) BroadcastHashJoin [codegen id : 9] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(71) Project [codegen id : 5] +(67) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(72) HashAggregate [codegen id : 5] +(68) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#40] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] -(73) RowToColumnar -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] - -(74) CometColumnarExchange +(69) Exchange Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] -Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(75) ColumnarToRow [codegen id : 6] -Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] - -(76) HashAggregate [codegen id : 6] +(70) HashAggregate [codegen id : 10] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#41] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(77) HashAggregate [codegen id : 6] +(71) HashAggregate [codegen id : 10] Input [1]: [netpaid#31] Keys: [] Functions [1]: [partial_avg(netpaid#31)] Aggregate Attributes [2]: [sum#42, count#43] Results [2]: [sum#44, count#45] -(78) RowToColumnar -Input [2]: [sum#44, count#45] - -(79) CometColumnarExchange -Input [2]: [sum#44, count#45] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(80) ColumnarToRow [codegen id : 7] +(72) Exchange Input [2]: [sum#44, count#45] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] -(81) HashAggregate [codegen id : 7] +(73) HashAggregate [codegen id : 11] Input [2]: [sum#44, count#45] Keys: [] Functions [1]: [avg(netpaid#31)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index 336a655002..8ebd45fd18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -1,112 +1,118 @@ -WholeStageCodegen (7) +WholeStageCodegen (11) Filter [paid] Subquery #1 - WholeStageCodegen (7) + WholeStageCodegen (11) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #9 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow + InputAdapter + Exchange #9 + WholeStageCodegen (10) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number] #4 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 + BroadcastExchange #11 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #3 + Exchange [ss_ticket_number,ss_item_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_color,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometFilter [i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ca_country,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt index 35c7a35d72..7eb30b6fb4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt @@ -1,46 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- * ColumnarToRow (40) - +- CometColumnarExchange (39) - +- RowToColumnar (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +TakeOrderedAndProject (40) ++- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (18) + : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) + : : : : +- ReusedExchange (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.store (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.item (31) (unknown) Scan parquet spark_catalog.default.store_sales @@ -116,7 +114,7 @@ Join condition: None Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -(16) ReusedExchange [Reuses operator id: 47] +(16) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#19] (17) BroadcastHashJoin [codegen id : 8] @@ -129,7 +127,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] -(19) ReusedExchange [Reuses operator id: 52] +(19) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#20] (20) BroadcastHashJoin [codegen id : 8] @@ -142,7 +140,7 @@ Join condition: None Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#20] -(22) ReusedExchange [Reuses operator id: 52] +(22) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#21] (23) BroadcastHashJoin [codegen id : 8] @@ -218,35 +216,29 @@ Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(Unscale Aggregate Attributes [3]: [sum#28, sum#29, sum#30] Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -(38) RowToColumnar +(38) Exchange Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(39) CometColumnarExchange -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) ColumnarToRow [codegen id : 9] -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] - -(41) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 9] Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#34, sum(UnscaledValue(sr_net_loss#11))#35, sum(UnscaledValue(cs_net_profit#16))#36] Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#34,17,2) AS store_sales_profit#37, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#35,17,2) AS store_returns_loss#38, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#36,17,2) AS catalog_sales_profit#39] -(42) TakeOrderedAndProject +(40) TakeOrderedAndProject Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) (unknown) Scan parquet spark_catalog.default.date_dim @@ -256,27 +248,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(42) CometFilter Input [3]: [d_date_sk#19, d_year#40, d_moy#41] Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 4)) AND (d_year#40 = 2001)) AND isnotnull(d_date_sk#19)) -(45) CometProject +(43) CometProject Input [3]: [d_date_sk#19, d_year#40, d_moy#41] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(47) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim @@ -286,18 +278,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter +(47) CometFilter Input [3]: [d_date_sk#20, d_year#42, d_moy#43] Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 4)) AND (d_moy#43 <= 10)) AND (d_year#42 = 2001)) AND isnotnull(d_date_sk#20)) -(50) CometProject +(48) CometProject Input [3]: [d_date_sk#20, d_year#42, d_moy#43] Arguments: [d_date_sk#20], [d_date_sk#20] -(51) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#20] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index 0418c4a7e7..eda7f6b64d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -1,78 +1,76 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] WholeStageCodegen (9) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] - Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 + WholeStageCodegen (8) + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] + Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index 192aacb4cb..55bae9c8c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -1,36 +1,34 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- * ColumnarToRow (30) - +- CometColumnarExchange (29) - +- RowToColumnar (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan parquet spark_catalog.default.promotion (20) +TakeOrderedAndProject (30) ++- * HashAggregate (29) + +- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * ColumnarToRow (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : +- ReusedExchange (11) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.item (14) + +- BroadcastExchange (24) + +- * ColumnarToRow (23) + +- CometProject (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.promotion (20) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -80,7 +78,7 @@ Join condition: None Output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] -(11) ReusedExchange [Reuses operator id: 37] +(11) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#14] (12) BroadcastHashJoin [codegen id : 5] @@ -160,35 +158,29 @@ Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_pr Aggregate Attributes [8]: [sum#20, count#21, sum#22, count#23, sum#24, count#25, sum#26, count#27] Results [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -(28) RowToColumnar +(28) Exchange Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) CometColumnarExchange -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) ColumnarToRow [codegen id : 6] -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] - -(31) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] Keys [1]: [i_item_id#16] Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] Aggregate Attributes [4]: [avg(cs_quantity#4)#36, avg(UnscaledValue(cs_list_price#5))#37, avg(UnscaledValue(cs_coupon_amt#7))#38, avg(UnscaledValue(cs_sales_price#6))#39] Results [5]: [i_item_id#16, avg(cs_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(cs_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(cs_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(cs_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] -(32) TakeOrderedAndProject +(30) TakeOrderedAndProject Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) (unknown) Scan parquet spark_catalog.default.date_dim @@ -198,18 +190,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(32) CometFilter Input [2]: [d_date_sk#14, d_year#44] Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) -(35) CometProject +(33) CometProject Input [2]: [d_date_sk#14, d_year#44] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(37) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index 67e6093dfc..7d38936244 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -1,54 +1,52 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (6) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index 0e3e6c9917..eca71fb097 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -1,36 +1,34 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- * ColumnarToRow (30) - +- CometColumnarExchange (29) - +- RowToColumnar (28) - +- * HashAggregate (27) - +- * Expand (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.store (14) - +- BroadcastExchange (23) - +- * ColumnarToRow (22) - +- CometFilter (21) - +- CometScan parquet spark_catalog.default.item (20) +TakeOrderedAndProject (30) ++- * HashAggregate (29) + +- Exchange (28) + +- * HashAggregate (27) + +- * Expand (26) + +- * Project (25) + +- * BroadcastHashJoin Inner BuildRight (24) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * ColumnarToRow (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : +- ReusedExchange (11) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.store (14) + +- BroadcastExchange (23) + +- * ColumnarToRow (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.item (20) (unknown) Scan parquet spark_catalog.default.store_sales @@ -80,7 +78,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(11) ReusedExchange [Reuses operator id: 37] +(11) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#14] (12) BroadcastHashJoin [codegen id : 5] @@ -160,35 +158,29 @@ Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_pr Aggregate Attributes [8]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] Results [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] -(28) RowToColumnar +(28) Exchange Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] +Arguments: hashpartitioning(i_item_id#19, s_state#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) CometColumnarExchange -Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] -Arguments: hashpartitioning(i_item_id#19, s_state#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) ColumnarToRow [codegen id : 6] -Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] - -(31) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] Keys [3]: [i_item_id#19, s_state#20, spark_grouping_id#21] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [4]: [avg(ss_quantity#4)#38, avg(UnscaledValue(ss_list_price#5))#39, avg(UnscaledValue(ss_coupon_amt#7))#40, avg(UnscaledValue(ss_sales_price#6))#41] Results [7]: [i_item_id#19, s_state#20, cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) AS g_state#42, avg(ss_quantity#4)#38 AS agg1#43, cast((avg(UnscaledValue(ss_list_price#5))#39 / 100.0) as decimal(11,6)) AS agg2#44, cast((avg(UnscaledValue(ss_coupon_amt#7))#40 / 100.0) as decimal(11,6)) AS agg3#45, cast((avg(UnscaledValue(ss_sales_price#6))#41 / 100.0) as decimal(11,6)) AS agg4#46] -(32) TakeOrderedAndProject +(30) TakeOrderedAndProject Input [7]: [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] Arguments: 100, [i_item_id#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST], [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) (unknown) Scan parquet spark_catalog.default.date_dim @@ -198,18 +190,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(32) CometFilter Input [2]: [d_date_sk#14, d_year#47] Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#14)) -(35) CometProject +(33) CometProject Input [2]: [d_date_sk#14, d_year#47] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(37) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index f57e4c443a..9d073ff67c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -1,54 +1,52 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (6) HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,s_state,spark_grouping_id] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + InputAdapter + Exchange [i_item_id,s_state,spark_grouping_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt index 16063ddeac..68590236c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt @@ -1,86 +1,74 @@ == Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (82) -:- * BroadcastNestedLoopJoin Inner BuildRight (68) -: :- * BroadcastNestedLoopJoin Inner BuildRight (54) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (40) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (26) -: : : : :- * ColumnarToRow (12) -: : : : : +- CometHashAggregate (11) -: : : : : +- CometColumnarExchange (10) -: : : : : +- RowToColumnar (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- * ColumnarToRow (6) -: : : : : +- CometColumnarExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (25) -: : : : +- * ColumnarToRow (24) -: : : : +- CometHashAggregate (23) -: : : : +- CometColumnarExchange (22) -: : : : +- RowToColumnar (21) -: : : : +- * HashAggregate (20) -: : : : +- * HashAggregate (19) -: : : : +- * ColumnarToRow (18) -: : : : +- CometColumnarExchange (17) -: : : : +- CometHashAggregate (16) -: : : : +- CometProject (15) -: : : : +- CometFilter (14) -: : : : +- CometScan parquet spark_catalog.default.store_sales (13) -: : : +- BroadcastExchange (39) -: : : +- * ColumnarToRow (38) -: : : +- CometHashAggregate (37) -: : : +- CometColumnarExchange (36) -: : : +- RowToColumnar (35) -: : : +- * HashAggregate (34) -: : : +- * HashAggregate (33) -: : : +- * ColumnarToRow (32) -: : : +- CometColumnarExchange (31) -: : : +- CometHashAggregate (30) -: : : +- CometProject (29) -: : : +- CometFilter (28) -: : : +- CometScan parquet spark_catalog.default.store_sales (27) -: : +- BroadcastExchange (53) -: : +- * ColumnarToRow (52) -: : +- CometHashAggregate (51) -: : +- CometColumnarExchange (50) -: : +- RowToColumnar (49) -: : +- * HashAggregate (48) -: : +- * HashAggregate (47) -: : +- * ColumnarToRow (46) -: : +- CometColumnarExchange (45) -: : +- CometHashAggregate (44) -: : +- CometProject (43) -: : +- CometFilter (42) -: : +- CometScan parquet spark_catalog.default.store_sales (41) -: +- BroadcastExchange (67) -: +- * ColumnarToRow (66) -: +- CometHashAggregate (65) -: +- CometColumnarExchange (64) -: +- RowToColumnar (63) -: +- * HashAggregate (62) -: +- * HashAggregate (61) -: +- * ColumnarToRow (60) -: +- CometColumnarExchange (59) -: +- CometHashAggregate (58) -: +- CometProject (57) -: +- CometFilter (56) -: +- CometScan parquet spark_catalog.default.store_sales (55) -+- BroadcastExchange (81) - +- * ColumnarToRow (80) - +- CometHashAggregate (79) - +- CometColumnarExchange (78) - +- RowToColumnar (77) - +- * HashAggregate (76) - +- * HashAggregate (75) - +- * ColumnarToRow (74) - +- CometColumnarExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometFilter (70) - +- CometScan parquet spark_catalog.default.store_sales (69) +* BroadcastNestedLoopJoin Inner BuildRight (70) +:- * BroadcastNestedLoopJoin Inner BuildRight (58) +: :- * BroadcastNestedLoopJoin Inner BuildRight (46) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (34) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (22) +: : : : :- * HashAggregate (10) +: : : : : +- Exchange (9) +: : : : : +- * HashAggregate (8) +: : : : : +- * HashAggregate (7) +: : : : : +- Exchange (6) +: : : : : +- * ColumnarToRow (5) +: : : : : +- CometHashAggregate (4) +: : : : : +- CometProject (3) +: : : : : +- CometFilter (2) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : +- BroadcastExchange (21) +: : : : +- * HashAggregate (20) +: : : : +- Exchange (19) +: : : : +- * HashAggregate (18) +: : : : +- * HashAggregate (17) +: : : : +- Exchange (16) +: : : : +- * ColumnarToRow (15) +: : : : +- CometHashAggregate (14) +: : : : +- CometProject (13) +: : : : +- CometFilter (12) +: : : : +- CometScan parquet spark_catalog.default.store_sales (11) +: : : +- BroadcastExchange (33) +: : : +- * HashAggregate (32) +: : : +- Exchange (31) +: : : +- * HashAggregate (30) +: : : +- * HashAggregate (29) +: : : +- Exchange (28) +: : : +- * ColumnarToRow (27) +: : : +- CometHashAggregate (26) +: : : +- CometProject (25) +: : : +- CometFilter (24) +: : : +- CometScan parquet spark_catalog.default.store_sales (23) +: : +- BroadcastExchange (45) +: : +- * HashAggregate (44) +: : +- Exchange (43) +: : +- * HashAggregate (42) +: : +- * HashAggregate (41) +: : +- Exchange (40) +: : +- * ColumnarToRow (39) +: : +- CometHashAggregate (38) +: : +- CometProject (37) +: : +- CometFilter (36) +: : +- CometScan parquet spark_catalog.default.store_sales (35) +: +- BroadcastExchange (57) +: +- * HashAggregate (56) +: +- Exchange (55) +: +- * HashAggregate (54) +: +- * HashAggregate (53) +: +- Exchange (52) +: +- * ColumnarToRow (51) +: +- CometHashAggregate (50) +: +- CometProject (49) +: +- CometFilter (48) +: +- CometScan parquet spark_catalog.default.store_sales (47) ++- BroadcastExchange (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * ColumnarToRow (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.store_sales (59) (unknown) Scan parquet spark_catalog.default.store_sales @@ -103,41 +91,37 @@ Input [1]: [ss_list_price#3] Keys [1]: [ss_list_price#3] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -(5) CometColumnarExchange +(5) ColumnarToRow [codegen id : 1] Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(6) ColumnarToRow [codegen id : 1] +(6) Exchange Input [4]: [ss_list_price#3, sum#6, count#7, count#8] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(7) HashAggregate [codegen id : 1] +(7) HashAggregate [codegen id : 2] Input [4]: [ss_list_price#3, sum#6, count#7, count#8] Keys [1]: [ss_list_price#3] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10] Results [4]: [ss_list_price#3, sum#6, count#7, count#8] -(8) HashAggregate [codegen id : 1] +(8) HashAggregate [codegen id : 2] Input [4]: [ss_list_price#3, sum#6, count#7, count#8] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] Results [4]: [sum#6, count#7, count#8, count#12] -(9) RowToColumnar +(9) Exchange Input [4]: [sum#6, count#7, count#8, count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(10) CometColumnarExchange -Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(11) CometHashAggregate +(10) HashAggregate [codegen id : 18] Input [4]: [sum#6, count#7, count#8, count#12] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] - -(12) ColumnarToRow [codegen id : 12] -Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#9 / 100.0) as decimal(11,6)) AS B1_LP#13, count(ss_list_price#3)#10 AS B1_CNT#14, count(ss_list_price#3)#11 AS B1_CNTD#15] (unknown) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] @@ -146,60 +130,56 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) -(15) CometProject +(13) CometProject Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] Arguments: [ss_list_price#18], [ss_list_price#18] -(16) CometHashAggregate +(14) CometHashAggregate Input [1]: [ss_list_price#18] Keys [1]: [ss_list_price#18] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] -(17) CometColumnarExchange +(15) ColumnarToRow [codegen id : 3] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) ColumnarToRow [codegen id : 2] +(16) Exchange Input [4]: [ss_list_price#18, sum#21, count#22, count#23] +Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(19) HashAggregate [codegen id : 2] +(17) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Keys [1]: [ss_list_price#18] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] Results [4]: [ss_list_price#18, sum#21, count#22, count#23] -(20) HashAggregate [codegen id : 2] +(18) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] Results [4]: [sum#21, count#22, count#23, count#27] -(21) RowToColumnar +(19) Exchange Input [4]: [sum#21, count#22, count#23, count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(22) CometColumnarExchange -Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(23) CometHashAggregate +(20) HashAggregate [codegen id : 5] Input [4]: [sum#21, count#22, count#23, count#27] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#18))#24 / 100.0) as decimal(11,6)) AS B2_LP#28, count(ss_list_price#18)#25 AS B2_CNT#29, count(ss_list_price#18)#26 AS B2_CNTD#30] -(24) ColumnarToRow [codegen id : 3] -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] - -(25) BroadcastExchange +(21) BroadcastExchange Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] Arguments: IdentityBroadcastMode, [plan_id=5] -(26) BroadcastNestedLoopJoin [codegen id : 12] +(22) BroadcastNestedLoopJoin [codegen id : 18] Join type: Inner Join condition: None @@ -210,60 +190,56 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct -(28) CometFilter +(24) CometFilter Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) -(29) CometProject +(25) CometProject Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] Arguments: [ss_list_price#33], [ss_list_price#33] -(30) CometHashAggregate +(26) CometHashAggregate Input [1]: [ss_list_price#33] Keys [1]: [ss_list_price#33] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] -(31) CometColumnarExchange +(27) ColumnarToRow [codegen id : 6] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(32) ColumnarToRow [codegen id : 4] +(28) Exchange Input [4]: [ss_list_price#33, sum#36, count#37, count#38] +Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(33) HashAggregate [codegen id : 4] +(29) HashAggregate [codegen id : 7] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys [1]: [ss_list_price#33] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] Results [4]: [ss_list_price#33, sum#36, count#37, count#38] -(34) HashAggregate [codegen id : 4] +(30) HashAggregate [codegen id : 7] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] Results [4]: [sum#36, count#37, count#38, count#42] -(35) RowToColumnar -Input [4]: [sum#36, count#37, count#38, count#42] - -(36) CometColumnarExchange +(31) Exchange Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(37) CometHashAggregate +(32) HashAggregate [codegen id : 8] Input [4]: [sum#36, count#37, count#38, count#42] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#33))#39 / 100.0) as decimal(11,6)) AS B3_LP#43, count(ss_list_price#33)#40 AS B3_CNT#44, count(ss_list_price#33)#41 AS B3_CNTD#45] -(38) ColumnarToRow [codegen id : 5] -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] - -(39) BroadcastExchange +(33) BroadcastExchange Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] Arguments: IdentityBroadcastMode, [plan_id=8] -(40) BroadcastNestedLoopJoin [codegen id : 12] +(34) BroadcastNestedLoopJoin [codegen id : 18] Join type: Inner Join condition: None @@ -274,60 +250,56 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct -(42) CometFilter +(36) CometFilter Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) -(43) CometProject +(37) CometProject Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] Arguments: [ss_list_price#48], [ss_list_price#48] -(44) CometHashAggregate +(38) CometHashAggregate Input [1]: [ss_list_price#48] Keys [1]: [ss_list_price#48] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] -(45) CometColumnarExchange +(39) ColumnarToRow [codegen id : 9] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(46) ColumnarToRow [codegen id : 6] +(40) Exchange Input [4]: [ss_list_price#48, sum#51, count#52, count#53] +Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(47) HashAggregate [codegen id : 6] +(41) HashAggregate [codegen id : 10] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys [1]: [ss_list_price#48] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] Results [4]: [ss_list_price#48, sum#51, count#52, count#53] -(48) HashAggregate [codegen id : 6] +(42) HashAggregate [codegen id : 10] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] Results [4]: [sum#51, count#52, count#53, count#57] -(49) RowToColumnar +(43) Exchange Input [4]: [sum#51, count#52, count#53, count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(50) CometColumnarExchange -Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(51) CometHashAggregate +(44) HashAggregate [codegen id : 11] Input [4]: [sum#51, count#52, count#53, count#57] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#48))#54 / 100.0) as decimal(11,6)) AS B4_LP#58, count(ss_list_price#48)#55 AS B4_CNT#59, count(ss_list_price#48)#56 AS B4_CNTD#60] -(52) ColumnarToRow [codegen id : 7] -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] - -(53) BroadcastExchange +(45) BroadcastExchange Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] Arguments: IdentityBroadcastMode, [plan_id=11] -(54) BroadcastNestedLoopJoin [codegen id : 12] +(46) BroadcastNestedLoopJoin [codegen id : 18] Join type: Inner Join condition: None @@ -338,60 +310,56 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct -(56) CometFilter +(48) CometFilter Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) -(57) CometProject +(49) CometProject Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] Arguments: [ss_list_price#63], [ss_list_price#63] -(58) CometHashAggregate +(50) CometHashAggregate Input [1]: [ss_list_price#63] Keys [1]: [ss_list_price#63] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] -(59) CometColumnarExchange +(51) ColumnarToRow [codegen id : 12] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(60) ColumnarToRow [codegen id : 8] +(52) Exchange Input [4]: [ss_list_price#63, sum#66, count#67, count#68] +Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(61) HashAggregate [codegen id : 8] +(53) HashAggregate [codegen id : 13] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys [1]: [ss_list_price#63] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] Results [4]: [ss_list_price#63, sum#66, count#67, count#68] -(62) HashAggregate [codegen id : 8] +(54) HashAggregate [codegen id : 13] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] Results [4]: [sum#66, count#67, count#68, count#72] -(63) RowToColumnar +(55) Exchange Input [4]: [sum#66, count#67, count#68, count#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(64) CometColumnarExchange -Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(65) CometHashAggregate +(56) HashAggregate [codegen id : 14] Input [4]: [sum#66, count#67, count#68, count#72] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#63))#69 / 100.0) as decimal(11,6)) AS B5_LP#73, count(ss_list_price#63)#70 AS B5_CNT#74, count(ss_list_price#63)#71 AS B5_CNTD#75] -(66) ColumnarToRow [codegen id : 9] -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] - -(67) BroadcastExchange +(57) BroadcastExchange Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] Arguments: IdentityBroadcastMode, [plan_id=14] -(68) BroadcastNestedLoopJoin [codegen id : 12] +(58) BroadcastNestedLoopJoin [codegen id : 18] Join type: Inner Join condition: None @@ -402,60 +370,56 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct -(70) CometFilter +(60) CometFilter Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) -(71) CometProject +(61) CometProject Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] Arguments: [ss_list_price#78], [ss_list_price#78] -(72) CometHashAggregate +(62) CometHashAggregate Input [1]: [ss_list_price#78] Keys [1]: [ss_list_price#78] Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] -(73) CometColumnarExchange +(63) ColumnarToRow [codegen id : 15] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(74) ColumnarToRow [codegen id : 10] +(64) Exchange Input [4]: [ss_list_price#78, sum#81, count#82, count#83] +Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(75) HashAggregate [codegen id : 10] +(65) HashAggregate [codegen id : 16] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys [1]: [ss_list_price#78] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] Results [4]: [ss_list_price#78, sum#81, count#82, count#83] -(76) HashAggregate [codegen id : 10] +(66) HashAggregate [codegen id : 16] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] Results [4]: [sum#81, count#82, count#83, count#87] -(77) RowToColumnar -Input [4]: [sum#81, count#82, count#83, count#87] - -(78) CometColumnarExchange +(67) Exchange Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] -(79) CometHashAggregate +(68) HashAggregate [codegen id : 17] Input [4]: [sum#81, count#82, count#83, count#87] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#78))#84 / 100.0) as decimal(11,6)) AS B6_LP#88, count(ss_list_price#78)#85 AS B6_CNT#89, count(ss_list_price#78)#86 AS B6_CNTD#90] -(80) ColumnarToRow [codegen id : 11] -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] - -(81) BroadcastExchange +(69) BroadcastExchange Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] Arguments: IdentityBroadcastMode, [plan_id=17] -(82) BroadcastNestedLoopJoin [codegen id : 12] +(70) BroadcastNestedLoopJoin [codegen id : 18] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt index e84abd7c1f..47a4fe2c7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt @@ -1,110 +1,110 @@ -WholeStageCodegen (12) +WholeStageCodegen (18) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - ColumnarToRow + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count] InputAdapter - CometHashAggregate [sum,count,count,count] - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_list_price] #2 + Exchange #1 + WholeStageCodegen (2) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometHashAggregate [ss_list_price] CometProject [ss_list_price] CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 - WholeStageCodegen (3) - ColumnarToRow + WholeStageCodegen (5) + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count] InputAdapter - CometHashAggregate [sum,count,count,count] - CometColumnarExchange #4 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_list_price] #5 + Exchange #4 + WholeStageCodegen (4) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter CometHashAggregate [ss_list_price] CometProject [ss_list_price] CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow + WholeStageCodegen (8) + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count] InputAdapter - CometHashAggregate [sum,count,count,count] - CometColumnarExchange #7 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_list_price] #8 + Exchange #7 + WholeStageCodegen (7) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #8 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter CometHashAggregate [ss_list_price] CometProject [ss_list_price] CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow + WholeStageCodegen (11) + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count] InputAdapter - CometHashAggregate [sum,count,count,count] - CometColumnarExchange #10 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_list_price] #11 + Exchange #10 + WholeStageCodegen (10) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #11 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter CometHashAggregate [ss_list_price] CometProject [ss_list_price] CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 - WholeStageCodegen (9) - ColumnarToRow + WholeStageCodegen (14) + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count] InputAdapter - CometHashAggregate [sum,count,count,count] - CometColumnarExchange #13 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_list_price] #14 + Exchange #13 + WholeStageCodegen (13) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #14 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter CometHashAggregate [ss_list_price] CometProject [ss_list_price] CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 - WholeStageCodegen (11) - ColumnarToRow + WholeStageCodegen (17) + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count] InputAdapter - CometHashAggregate [sum,count,count,count] - CometColumnarExchange #16 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_list_price] #17 + Exchange #16 + WholeStageCodegen (16) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #17 + WholeStageCodegen (15) + ColumnarToRow + InputAdapter CometHashAggregate [ss_list_price] CometProject [ss_list_price] CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt index 3a3000118a..e74f10d25a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt @@ -1,46 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- * ColumnarToRow (40) - +- CometColumnarExchange (39) - +- RowToColumnar (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +TakeOrderedAndProject (40) ++- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (18) + : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) + : : : : +- ReusedExchange (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.store (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.item (31) (unknown) Scan parquet spark_catalog.default.store_sales @@ -116,7 +114,7 @@ Join condition: None Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -(16) ReusedExchange [Reuses operator id: 47] +(16) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#19] (17) BroadcastHashJoin [codegen id : 8] @@ -129,7 +127,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -(19) ReusedExchange [Reuses operator id: 52] +(19) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#20] (20) BroadcastHashJoin [codegen id : 8] @@ -142,7 +140,7 @@ Join condition: None Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] -(22) ReusedExchange [Reuses operator id: 57] +(22) ReusedExchange [Reuses operator id: 55] Output [1]: [d_date_sk#21] (23) BroadcastHashJoin [codegen id : 8] @@ -218,35 +216,29 @@ Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), Aggregate Attributes [3]: [sum#28, sum#29, sum#30] Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -(38) RowToColumnar +(38) Exchange Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(39) CometColumnarExchange -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) ColumnarToRow [codegen id : 9] -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] - -(41) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 9] Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] Aggregate Attributes [3]: [sum(ss_quantity#5)#34, sum(sr_return_quantity#11)#35, sum(cs_quantity#16)#36] Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum(ss_quantity#5)#34 AS store_sales_quantity#37, sum(sr_return_quantity#11)#35 AS store_returns_quantity#38, sum(cs_quantity#16)#36 AS catalog_sales_quantity#39] -(42) TakeOrderedAndProject +(40) TakeOrderedAndProject Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) (unknown) Scan parquet spark_catalog.default.date_dim @@ -256,27 +248,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(42) CometFilter Input [3]: [d_date_sk#19, d_year#40, d_moy#41] Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 9)) AND (d_year#40 = 1999)) AND isnotnull(d_date_sk#19)) -(45) CometProject +(43) CometProject Input [3]: [d_date_sk#19, d_year#40, d_moy#41] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(47) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim @@ -286,27 +278,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter +(47) CometFilter Input [3]: [d_date_sk#20, d_year#42, d_moy#43] Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 9)) AND (d_moy#43 <= 12)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#20)) -(50) CometProject +(48) CometProject Input [3]: [d_date_sk#20, d_year#42, d_moy#43] Arguments: [d_date_sk#20], [d_date_sk#20] -(51) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#20] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 -BroadcastExchange (57) -+- * ColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan parquet spark_catalog.default.date_dim (53) +BroadcastExchange (55) ++- * ColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) (unknown) Scan parquet spark_catalog.default.date_dim @@ -316,18 +308,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(54) CometFilter +(52) CometFilter Input [2]: [d_date_sk#21, d_year#44] Condition : (d_year#44 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) -(55) CometProject +(53) CometProject Input [2]: [d_date_sk#21, d_year#44] Arguments: [d_date_sk#21], [d_date_sk#21] -(56) ColumnarToRow [codegen id : 1] +(54) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#21] -(57) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt index 9de3a02a39..68a127d357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt @@ -1,85 +1,83 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] WholeStageCodegen (9) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 + WholeStageCodegen (8) + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] + Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #6 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + ReusedExchange [d_date_sk] #6 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt index dab78aab5a..22b8681f2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt @@ -1,27 +1,25 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- * ColumnarToRow (21) - +- CometColumnarExchange (20) - +- RowToColumnar (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- Exchange (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) (unknown) Scan parquet spark_catalog.default.date_dim @@ -110,24 +108,18 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -(19) RowToColumnar +(19) Exchange Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) CometColumnarExchange -Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) ColumnarToRow [codegen id : 4] -Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] - -(22) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 4] Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum_agg#16] -(23) TakeOrderedAndProject +(21) TakeOrderedAndProject Input [4]: [d_year#2, brand_id#14, brand#15, sum_agg#16] Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, sum_agg#16] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt index 191bf4c810..3946c0cd8f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt @@ -1,33 +1,31 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (4) HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [d_year,i_brand,i_brand_id] #1 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + Exchange [d_year,i_brand,i_brand_id] #1 + WholeStageCodegen (3) + HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year] + CometFilter [d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index 2b3ab365c0..5da800fe40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt @@ -1,59 +1,53 @@ == Physical Plan == -TakeOrderedAndProject (55) -+- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (47) - : +- * BroadcastHashJoin Inner BuildRight (46) - : :- * Project (41) - : : +- * BroadcastHashJoin Inner BuildRight (40) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * ColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- RowToColumnar (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (39) - : : +- * Filter (38) - : : +- * HashAggregate (37) - : : +- * ColumnarToRow (36) - : : +- CometColumnarExchange (35) - : : +- RowToColumnar (34) - : : +- * HashAggregate (33) - : : +- * HashAggregate (32) - : : +- * ColumnarToRow (31) - : : +- CometColumnarExchange (30) - : : +- RowToColumnar (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.web_returns (19) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (45) - : +- * ColumnarToRow (44) - : +- CometFilter (43) - : +- CometScan parquet spark_catalog.default.customer (42) - +- BroadcastExchange (52) - +- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.customer_address (48) +TakeOrderedAndProject (49) ++- * Project (48) + +- * BroadcastHashJoin Inner BuildRight (47) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan parquet spark_catalog.default.customer_address (7) + : : +- BroadcastExchange (33) + : : +- * Filter (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * HashAggregate (28) + : : +- Exchange (27) + : : +- * HashAggregate (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.web_returns (17) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : +- BroadcastExchange (39) + : +- * ColumnarToRow (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.customer (36) + +- BroadcastExchange (46) + +- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.customer_address (42) (unknown) Scan parquet spark_catalog.default.web_returns @@ -71,7 +65,7 @@ Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_custom (3) ColumnarToRow [codegen id : 3] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -(4) ReusedExchange [Reuses operator id: 60] +(4) ReusedExchange [Reuses operator id: 54] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -119,24 +113,18 @@ Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum#9] Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] -(14) RowToColumnar +(14) Exchange Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 11] -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] - -(17) HashAggregate [codegen id : 11] +(15) HashAggregate [codegen id : 11] Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] Keys [2]: [wr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#11] Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] -(18) Filter [codegen id : 11] +(16) Filter [codegen id : 11] Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] Condition : isnotnull(ctr_total_return#14) @@ -148,102 +136,90 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(20) CometFilter +(18) CometFilter Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Condition : isnotnull(wr_returning_addr_sk#2) -(21) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 6] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -(22) ReusedExchange [Reuses operator id: 60] +(20) ReusedExchange [Reuses operator id: 54] Output [1]: [d_date_sk#6] -(23) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 6] Left keys [1]: [wr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(22) Project [codegen id : 6] Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] -(25) ReusedExchange [Reuses operator id: 10] +(23) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#7, ca_state#8] -(26) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [wr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(25) Project [codegen id : 6] Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#8] -(28) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 6] Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] Keys [2]: [wr_returning_customer_sk#1, ca_state#8] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum#16] Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] -(29) RowToColumnar -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] - -(30) CometColumnarExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(31) ColumnarToRow [codegen id : 7] +(27) Exchange Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) HashAggregate [codegen id : 7] +(28) HashAggregate [codegen id : 7] Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] Keys [2]: [wr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#11] Results [2]: [ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] -(33) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 7] Input [2]: [ctr_state#13, ctr_total_return#14] Keys [1]: [ctr_state#13] Functions [1]: [partial_avg(ctr_total_return#14)] Aggregate Attributes [2]: [sum#18, count#19] Results [3]: [ctr_state#13, sum#20, count#21] -(34) RowToColumnar -Input [3]: [ctr_state#13, sum#20, count#21] - -(35) CometColumnarExchange +(30) Exchange Input [3]: [ctr_state#13, sum#20, count#21] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(36) ColumnarToRow [codegen id : 8] -Input [3]: [ctr_state#13, sum#20, count#21] - -(37) HashAggregate [codegen id : 8] +(31) HashAggregate [codegen id : 8] Input [3]: [ctr_state#13, sum#20, count#21] Keys [1]: [ctr_state#13] Functions [1]: [avg(ctr_total_return#14)] Aggregate Attributes [1]: [avg(ctr_total_return#14)#22] Results [2]: [(avg(ctr_total_return#14)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#13 AS ctr_state#13#24] -(38) Filter [codegen id : 8] +(32) Filter [codegen id : 8] Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) -(39) BroadcastExchange +(33) BroadcastExchange Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(40) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#13] Right keys [1]: [ctr_state#13#24] Join type: Inner Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) -(41) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [2]: [ctr_customer_sk#12, ctr_total_return#14] Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] @@ -254,24 +230,24 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(43) CometFilter +(37) CometFilter Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) -(44) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 9] Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] -(45) BroadcastExchange +(39) BroadcastExchange Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 11] +(40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#12] Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(47) Project [codegen id : 11] +(41) Project [codegen id : 11] Output [14]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] @@ -282,43 +258,43 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(49) CometFilter +(43) CometFilter Input [2]: [ca_address_sk#39, ca_state#40] Condition : ((isnotnull(ca_state#40) AND (ca_state#40 = GA)) AND isnotnull(ca_address_sk#39)) -(50) CometProject +(44) CometProject Input [2]: [ca_address_sk#39, ca_state#40] Arguments: [ca_address_sk#39], [ca_address_sk#39] -(51) ColumnarToRow [codegen id : 10] +(45) ColumnarToRow [codegen id : 10] Input [1]: [ca_address_sk#39] -(52) BroadcastExchange +(46) BroadcastExchange Input [1]: [ca_address_sk#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(53) BroadcastHashJoin [codegen id : 11] +(47) BroadcastHashJoin [codegen id : 11] Left keys [1]: [c_current_addr_sk#27] Right keys [1]: [ca_address_sk#39] Join type: Inner Join condition: None -(54) Project [codegen id : 11] +(48) Project [codegen id : 11] Output [13]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] Input [15]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ca_address_sk#39] -(55) TakeOrderedAndProject +(49) TakeOrderedAndProject Input [13]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] Arguments: 100, [c_customer_id#26 ASC NULLS FIRST, c_salutation#28 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, c_last_name#30 ASC NULLS FIRST, c_preferred_cust_flag#31 ASC NULLS FIRST, c_birth_day#32 ASC NULLS FIRST, c_birth_month#33 ASC NULLS FIRST, c_birth_year#34 ASC NULLS FIRST, c_birth_country#35 ASC NULLS FIRST, c_login#36 ASC NULLS FIRST, c_email_address#37 ASC NULLS FIRST, c_last_review_date#38 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (60) -+- * ColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.date_dim (56) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) (unknown) Scan parquet spark_catalog.default.date_dim @@ -328,21 +304,21 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter +(51) CometFilter Input [2]: [d_date_sk#6, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#6)) -(58) CometProject +(52) CometProject Input [2]: [d_date_sk#6, d_year#41] Arguments: [d_date_sk#6], [d_date_sk#6] -(59) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(60) BroadcastExchange +(54) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index d6458b01ba..c70f33da33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -8,68 +8,62 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] - CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 + InputAdapter + Exchange [wr_returning_customer_sk,ca_state] #1 + WholeStageCodegen (3) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 WholeStageCodegen (8) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #5 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [wr_returning_customer_sk,ca_state] #6 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [wr_returning_addr_sk] - CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 + InputAdapter + Exchange [ctr_state] #5 + WholeStageCodegen (7) + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [wr_returning_customer_sk,ca_state] #6 + WholeStageCodegen (6) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_returning_addr_sk] + CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 WholeStageCodegen (9) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index da7d8fd881..7b1f7bb5db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt @@ -1,108 +1,94 @@ == Physical Plan == -* ColumnarToRow (104) -+- CometSort (103) - +- CometColumnarExchange (102) - +- RowToColumnar (101) - +- * Project (100) - +- * BroadcastHashJoin Inner BuildRight (99) - :- * Project (83) - : +- * BroadcastHashJoin Inner BuildRight (82) - : :- * BroadcastHashJoin Inner BuildRight (66) - : : :- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * HashAggregate (17) - : : : : : +- * ColumnarToRow (16) - : : : : : +- CometColumnarExchange (15) - : : : : : +- RowToColumnar (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (6) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- ReusedExchange (4) - : : : : : +- BroadcastExchange (10) - : : : : : +- * ColumnarToRow (9) - : : : : : +- CometFilter (8) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : : : +- BroadcastExchange (32) - : : : : +- * HashAggregate (31) - : : : : +- * ColumnarToRow (30) - : : : : +- CometColumnarExchange (29) - : : : : +- RowToColumnar (28) - : : : : +- * HashAggregate (27) - : : : : +- * Project (26) - : : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : : :- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * ColumnarToRow (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (18) - : : : : : +- ReusedExchange (21) - : : : : +- ReusedExchange (24) - : : : +- BroadcastExchange (48) - : : : +- * HashAggregate (47) - : : : +- * ColumnarToRow (46) - : : : +- CometColumnarExchange (45) - : : : +- RowToColumnar (44) - : : : +- * HashAggregate (43) - : : : +- * Project (42) - : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : :- * Project (39) - : : : : +- * BroadcastHashJoin Inner BuildRight (38) - : : : : :- * ColumnarToRow (36) - : : : : : +- CometFilter (35) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (34) - : : : : +- ReusedExchange (37) - : : : +- ReusedExchange (40) - : : +- BroadcastExchange (65) - : : +- * HashAggregate (64) - : : +- * ColumnarToRow (63) - : : +- CometColumnarExchange (62) - : : +- RowToColumnar (61) - : : +- * HashAggregate (60) - : : +- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Project (56) - : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : :- * ColumnarToRow (53) - : : : : +- CometFilter (52) - : : : : +- CometScan parquet spark_catalog.default.web_sales (51) - : : : +- ReusedExchange (54) - : : +- ReusedExchange (57) - : +- BroadcastExchange (81) - : +- * HashAggregate (80) - : +- * ColumnarToRow (79) - : +- CometColumnarExchange (78) - : +- RowToColumnar (77) - : +- * HashAggregate (76) - : +- * Project (75) - : +- * BroadcastHashJoin Inner BuildRight (74) - : :- * Project (72) - : : +- * BroadcastHashJoin Inner BuildRight (71) - : : :- * ColumnarToRow (69) - : : : +- CometFilter (68) - : : : +- CometScan parquet spark_catalog.default.web_sales (67) - : : +- ReusedExchange (70) - : +- ReusedExchange (73) - +- BroadcastExchange (98) - +- * HashAggregate (97) - +- * ColumnarToRow (96) - +- CometColumnarExchange (95) - +- RowToColumnar (94) - +- * HashAggregate (93) - +- * Project (92) - +- * BroadcastHashJoin Inner BuildRight (91) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * ColumnarToRow (86) - : : +- CometFilter (85) - : : +- CometScan parquet spark_catalog.default.web_sales (84) - : +- ReusedExchange (87) - +- ReusedExchange (90) +* Sort (90) ++- Exchange (89) + +- * Project (88) + +- * BroadcastHashJoin Inner BuildRight (87) + :- * Project (73) + : +- * BroadcastHashJoin Inner BuildRight (72) + : :- * BroadcastHashJoin Inner BuildRight (58) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * BroadcastHashJoin Inner BuildRight (29) + : : : : :- * HashAggregate (15) + : : : : : +- Exchange (14) + : : : : : +- * HashAggregate (13) + : : : : : +- * Project (12) + : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : :- * Project (6) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- ReusedExchange (4) + : : : : : +- BroadcastExchange (10) + : : : : : +- * ColumnarToRow (9) + : : : : : +- CometFilter (8) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (7) + : : : : +- BroadcastExchange (28) + : : : : +- * HashAggregate (27) + : : : : +- Exchange (26) + : : : : +- * HashAggregate (25) + : : : : +- * Project (24) + : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : :- * Project (21) + : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : :- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (16) + : : : : : +- ReusedExchange (19) + : : : : +- ReusedExchange (22) + : : : +- BroadcastExchange (42) + : : : +- * HashAggregate (41) + : : : +- Exchange (40) + : : : +- * HashAggregate (39) + : : : +- * Project (38) + : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : :- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * ColumnarToRow (32) + : : : : : +- CometFilter (31) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (30) + : : : : +- ReusedExchange (33) + : : : +- ReusedExchange (36) + : : +- BroadcastExchange (57) + : : +- * HashAggregate (56) + : : +- Exchange (55) + : : +- * HashAggregate (54) + : : +- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * ColumnarToRow (47) + : : : : +- CometFilter (46) + : : : : +- CometScan parquet spark_catalog.default.web_sales (45) + : : : +- ReusedExchange (48) + : : +- ReusedExchange (51) + : +- BroadcastExchange (71) + : +- * HashAggregate (70) + : +- Exchange (69) + : +- * HashAggregate (68) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * ColumnarToRow (61) + : : : +- CometFilter (60) + : : : +- CometScan parquet spark_catalog.default.web_sales (59) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + +- BroadcastExchange (86) + +- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Project (82) + +- * BroadcastHashJoin Inner BuildRight (81) + :- * Project (79) + : +- * BroadcastHashJoin Inner BuildRight (78) + : :- * ColumnarToRow (76) + : : +- CometFilter (75) + : : +- CometScan parquet spark_catalog.default.web_sales (74) + : +- ReusedExchange (77) + +- ReusedExchange (80) (unknown) Scan parquet spark_catalog.default.store_sales @@ -120,7 +106,7 @@ Condition : isnotnull(ss_addr_sk#1) (3) ColumnarToRow [codegen id : 3] Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 108] +(4) ReusedExchange [Reuses operator id: 94] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] (5) BroadcastHashJoin [codegen id : 3] @@ -168,17 +154,11 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#10] Results [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -(14) RowToColumnar +(14) Exchange Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] +Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 24] -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] - -(17) HashAggregate [codegen id : 24] +(15) HashAggregate [codegen id : 24] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] @@ -193,68 +173,62 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] Condition : isnotnull(ss_addr_sk#14) -(20) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 6] Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -(21) ReusedExchange [Reuses operator id: 112] +(19) ReusedExchange [Reuses operator id: 98] Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] -(22) BroadcastHashJoin [codegen id : 6] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#16] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(23) Project [codegen id : 6] +(21) Project [codegen id : 6] Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] -(24) ReusedExchange [Reuses operator id: 10] +(22) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#21, ca_county#22] -(25) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_addr_sk#14] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] -(27) HashAggregate [codegen id : 6] +(25) HashAggregate [codegen id : 6] Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] Keys [3]: [ca_county#22, d_qoy#20, d_year#19] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] Aggregate Attributes [1]: [sum#23] Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -(28) RowToColumnar -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] - -(29) CometColumnarExchange -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) ColumnarToRow [codegen id : 7] +(26) Exchange Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] +Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(31) HashAggregate [codegen id : 7] +(27) HashAggregate [codegen id : 7] Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] Keys [3]: [ca_county#22, d_qoy#20, d_year#19] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#15))#12] Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#12,17,2) AS store_sales#25] -(32) BroadcastExchange +(28) BroadcastExchange Input [2]: [ca_county#22, store_sales#25] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 24] +(29) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#22] Join type: Inner @@ -268,74 +242,68 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Condition : isnotnull(ss_addr_sk#26) -(36) ColumnarToRow [codegen id : 10] +(32) ColumnarToRow [codegen id : 10] Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -(37) ReusedExchange [Reuses operator id: 116] +(33) ReusedExchange [Reuses operator id: 102] Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] -(38) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#28] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(39) Project [codegen id : 10] +(35) Project [codegen id : 10] Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] -(40) ReusedExchange [Reuses operator id: 10] +(36) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#33, ca_county#34] -(41) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_addr_sk#26] Right keys [1]: [ca_address_sk#33] Join type: Inner Join condition: None -(42) Project [codegen id : 10] +(38) Project [codegen id : 10] Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] -(43) HashAggregate [codegen id : 10] +(39) HashAggregate [codegen id : 10] Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] Keys [3]: [ca_county#34, d_qoy#32, d_year#31] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] Aggregate Attributes [1]: [sum#35] Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -(44) RowToColumnar +(40) Exchange Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] +Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(45) CometColumnarExchange -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(46) ColumnarToRow [codegen id : 11] -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] - -(47) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 11] Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] Keys [3]: [ca_county#34, d_qoy#32, d_year#31] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#12] Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#12,17,2) AS store_sales#37] -(48) BroadcastExchange +(42) BroadcastExchange Input [2]: [ca_county#34, store_sales#37] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 24] +(43) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#22] Right keys [1]: [ca_county#34] Join type: Inner Join condition: None -(50) Project [codegen id : 24] +(44) Project [codegen id : 24] Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] @@ -347,68 +315,62 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Condition : isnotnull(ws_bill_addr_sk#38) -(53) ColumnarToRow [codegen id : 14] +(47) ColumnarToRow [codegen id : 14] Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -(54) ReusedExchange [Reuses operator id: 108] +(48) ReusedExchange [Reuses operator id: 94] Output [3]: [d_date_sk#42, d_year#43, d_qoy#44] -(55) BroadcastHashJoin [codegen id : 14] +(49) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_sold_date_sk#40] Right keys [1]: [d_date_sk#42] Join type: Inner Join condition: None -(56) Project [codegen id : 14] +(50) Project [codegen id : 14] Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44] Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#42, d_year#43, d_qoy#44] -(57) ReusedExchange [Reuses operator id: 10] +(51) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#45, ca_county#46] -(58) BroadcastHashJoin [codegen id : 14] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_bill_addr_sk#38] Right keys [1]: [ca_address_sk#45] Join type: Inner Join condition: None -(59) Project [codegen id : 14] +(53) Project [codegen id : 14] Output [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_address_sk#45, ca_county#46] -(60) HashAggregate [codegen id : 14] +(54) HashAggregate [codegen id : 14] Input [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] Keys [3]: [ca_county#46, d_qoy#44, d_year#43] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] Aggregate Attributes [1]: [sum#47] Results [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] -(61) RowToColumnar -Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] - -(62) CometColumnarExchange -Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] -Arguments: hashpartitioning(ca_county#46, d_qoy#44, d_year#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(63) ColumnarToRow [codegen id : 15] +(55) Exchange Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] +Arguments: hashpartitioning(ca_county#46, d_qoy#44, d_year#43, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(64) HashAggregate [codegen id : 15] +(56) HashAggregate [codegen id : 15] Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] Keys [3]: [ca_county#46, d_qoy#44, d_year#43] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#49] Results [2]: [ca_county#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#49,17,2) AS web_sales#50] -(65) BroadcastExchange +(57) BroadcastExchange Input [2]: [ca_county#46, web_sales#50] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(66) BroadcastHashJoin [codegen id : 24] +(58) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#46] Join type: Inner @@ -422,74 +384,68 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(68) CometFilter +(60) CometFilter Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] Condition : isnotnull(ws_bill_addr_sk#51) -(69) ColumnarToRow [codegen id : 18] +(61) ColumnarToRow [codegen id : 18] Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] -(70) ReusedExchange [Reuses operator id: 112] +(62) ReusedExchange [Reuses operator id: 98] Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] -(71) BroadcastHashJoin [codegen id : 18] +(63) BroadcastHashJoin [codegen id : 18] Left keys [1]: [ws_sold_date_sk#53] Right keys [1]: [d_date_sk#55] Join type: Inner Join condition: None -(72) Project [codegen id : 18] +(64) Project [codegen id : 18] Output [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57] Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56, d_qoy#57] -(73) ReusedExchange [Reuses operator id: 10] +(65) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#58, ca_county#59] -(74) BroadcastHashJoin [codegen id : 18] +(66) BroadcastHashJoin [codegen id : 18] Left keys [1]: [ws_bill_addr_sk#51] Right keys [1]: [ca_address_sk#58] Join type: Inner Join condition: None -(75) Project [codegen id : 18] +(67) Project [codegen id : 18] Output [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_address_sk#58, ca_county#59] -(76) HashAggregate [codegen id : 18] +(68) HashAggregate [codegen id : 18] Input [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] Keys [3]: [ca_county#59, d_qoy#57, d_year#56] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#52))] Aggregate Attributes [1]: [sum#60] Results [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] -(77) RowToColumnar -Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] - -(78) CometColumnarExchange -Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] -Arguments: hashpartitioning(ca_county#59, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(79) ColumnarToRow [codegen id : 19] +(69) Exchange Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] +Arguments: hashpartitioning(ca_county#59, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(80) HashAggregate [codegen id : 19] +(70) HashAggregate [codegen id : 19] Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] Keys [3]: [ca_county#59, d_qoy#57, d_year#56] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#52))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#52))#49] Results [2]: [ca_county#59, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#52))#49,17,2) AS web_sales#62] -(81) BroadcastExchange +(71) BroadcastExchange Input [2]: [ca_county#59, web_sales#62] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] -(82) BroadcastHashJoin [codegen id : 24] +(72) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#46] Right keys [1]: [ca_county#59] Join type: Inner Join condition: (CASE WHEN (web_sales#50 > 0.00) THEN (web_sales#62 / web_sales#50) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) -(83) Project [codegen id : 24] +(73) Project [codegen id : 24] Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62] Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, ca_county#59, web_sales#62] @@ -501,98 +457,86 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(85) CometFilter +(75) CometFilter Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] Condition : isnotnull(ws_bill_addr_sk#63) -(86) ColumnarToRow [codegen id : 22] +(76) ColumnarToRow [codegen id : 22] Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] -(87) ReusedExchange [Reuses operator id: 116] +(77) ReusedExchange [Reuses operator id: 102] Output [3]: [d_date_sk#67, d_year#68, d_qoy#69] -(88) BroadcastHashJoin [codegen id : 22] +(78) BroadcastHashJoin [codegen id : 22] Left keys [1]: [ws_sold_date_sk#65] Right keys [1]: [d_date_sk#67] Join type: Inner Join condition: None -(89) Project [codegen id : 22] +(79) Project [codegen id : 22] Output [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69] Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68, d_qoy#69] -(90) ReusedExchange [Reuses operator id: 10] +(80) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#70, ca_county#71] -(91) BroadcastHashJoin [codegen id : 22] +(81) BroadcastHashJoin [codegen id : 22] Left keys [1]: [ws_bill_addr_sk#63] Right keys [1]: [ca_address_sk#70] Join type: Inner Join condition: None -(92) Project [codegen id : 22] +(82) Project [codegen id : 22] Output [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_address_sk#70, ca_county#71] -(93) HashAggregate [codegen id : 22] +(83) HashAggregate [codegen id : 22] Input [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] Keys [3]: [ca_county#71, d_qoy#69, d_year#68] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] Aggregate Attributes [1]: [sum#72] Results [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] -(94) RowToColumnar -Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] - -(95) CometColumnarExchange +(84) Exchange Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] -Arguments: hashpartitioning(ca_county#71, d_qoy#69, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +Arguments: hashpartitioning(ca_county#71, d_qoy#69, d_year#68, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(96) ColumnarToRow [codegen id : 23] -Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] - -(97) HashAggregate [codegen id : 23] +(85) HashAggregate [codegen id : 23] Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] Keys [3]: [ca_county#71, d_qoy#69, d_year#68] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#64))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#64))#49] Results [2]: [ca_county#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#49,17,2) AS web_sales#74] -(98) BroadcastExchange +(86) BroadcastExchange Input [2]: [ca_county#71, web_sales#74] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] -(99) BroadcastHashJoin [codegen id : 24] +(87) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#46] Right keys [1]: [ca_county#71] Join type: Inner Join condition: (CASE WHEN (web_sales#62 > 0.00) THEN (web_sales#74 / web_sales#62) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) -(100) Project [codegen id : 24] +(88) Project [codegen id : 24] Output [6]: [ca_county#9, d_year#6, (web_sales#62 / web_sales#50) AS web_q1_q2_increase#75, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#76, (web_sales#74 / web_sales#62) AS web_q2_q3_increase#77, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#78] Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62, ca_county#71, web_sales#74] -(101) RowToColumnar -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] - -(102) CometColumnarExchange +(89) Exchange Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(103) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78], [ca_county#9 ASC NULLS FIRST] - -(104) ColumnarToRow [codegen id : 25] +(90) Sort [codegen id : 25] Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] +Arguments: [ca_county#9 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (108) -+- * ColumnarToRow (107) - +- CometFilter (106) - +- CometScan parquet spark_catalog.default.date_dim (105) +BroadcastExchange (94) ++- * ColumnarToRow (93) + +- CometFilter (92) + +- CometScan parquet spark_catalog.default.date_dim (91) (unknown) Scan parquet spark_catalog.default.date_dim @@ -602,22 +546,22 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(106) CometFilter +(92) CometFilter Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(107) ColumnarToRow [codegen id : 1] +(93) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(108) BroadcastExchange +(94) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (112) -+- * ColumnarToRow (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 +BroadcastExchange (98) ++- * ColumnarToRow (97) + +- CometFilter (96) + +- CometScan parquet spark_catalog.default.date_dim (95) (unknown) Scan parquet spark_catalog.default.date_dim @@ -627,22 +571,22 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(110) CometFilter +(96) CometFilter Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) -(111) ColumnarToRow [codegen id : 1] +(97) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -(112) BroadcastExchange +(98) BroadcastExchange Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -Subquery:3 Hosting operator id = 34 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 -BroadcastExchange (116) -+- * ColumnarToRow (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 +BroadcastExchange (102) ++- * ColumnarToRow (101) + +- CometFilter (100) + +- CometScan parquet spark_catalog.default.date_dim (99) (unknown) Scan parquet spark_catalog.default.date_dim @@ -652,21 +596,21 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter +(100) CometFilter Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) -(115) ColumnarToRow [codegen id : 1] +(101) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -(116) BroadcastExchange +(102) BroadcastExchange Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:4 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#17 +Subquery:5 Hosting operator id = 59 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#17 -Subquery:6 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#29 +Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#29 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index e765d9ae12..f4bf6a89d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -1,173 +1,159 @@ WholeStageCodegen (25) - ColumnarToRow + Sort [ca_county] InputAdapter - CometSort [ca_county] - CometColumnarExchange [ca_county] #1 - RowToColumnar - WholeStageCodegen (24) - Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] + Exchange [ca_county] #1 + WholeStageCodegen (24) + Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county] + Project [ca_county,d_year,store_sales,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #6 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ca_county,ca_county] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #2 + WholeStageCodegen (3) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #9 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #12 - RowToColumnar - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #14 - RowToColumnar - WholeStageCodegen (18) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #6 + WholeStageCodegen (6) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_addr_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + CometFilter [ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk,d_year,d_qoy] #7 InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #16 - RowToColumnar - WholeStageCodegen (22) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #9 + WholeStageCodegen (10) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #10 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - CometFilter [ws_bill_addr_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk,d_year,d_qoy] #10 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #12 + WholeStageCodegen (14) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 + CometFilter [ws_bill_addr_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (19) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #14 + WholeStageCodegen (18) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #7 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (23) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #16 + WholeStageCodegen (22) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #10 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 30cc61779d..e3c659e278 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -1,37 +1,33 @@ == Physical Plan == -* HashAggregate (33) -+- * ColumnarToRow (32) - +- CometColumnarExchange (31) - +- RowToColumnar (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (23) - : +- * Filter (22) - : +- * HashAggregate (21) - : +- * ColumnarToRow (20) - : +- CometColumnarExchange (19) - : +- RowToColumnar (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * ColumnarToRow (13) - : : +- CometFilter (12) - : : +- CometScan parquet spark_catalog.default.catalog_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (26) +* HashAggregate (29) ++- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (23) + : +- * BroadcastHashJoin Inner BuildRight (22) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.item (4) + : +- BroadcastExchange (21) + : +- * Filter (20) + : +- * HashAggregate (19) + : +- Exchange (18) + : +- * HashAggregate (17) + : +- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * ColumnarToRow (13) + : : +- CometFilter (12) + : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : +- ReusedExchange (14) + +- ReusedExchange (24) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -96,7 +92,7 @@ Condition : isnotnull(cs_item_sk#7) (13) ColumnarToRow [codegen id : 3] Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -(14) ReusedExchange [Reuses operator id: 38] +(14) ReusedExchange [Reuses operator id: 34] Output [1]: [d_date_sk#11] (15) BroadcastHashJoin [codegen id : 3] @@ -116,72 +112,60 @@ Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] Aggregate Attributes [2]: [sum#12, count#13] Results [3]: [cs_item_sk#7, sum#14, count#15] -(18) RowToColumnar +(18) Exchange Input [3]: [cs_item_sk#7, sum#14, count#15] +Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(19) CometColumnarExchange -Input [3]: [cs_item_sk#7, sum#14, count#15] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(20) ColumnarToRow [codegen id : 4] -Input [3]: [cs_item_sk#7, sum#14, count#15] - -(21) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 4] Input [3]: [cs_item_sk#7, sum#14, count#15] Keys [1]: [cs_item_sk#7] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#16] Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] -(22) Filter [codegen id : 4] +(20) Filter [codegen id : 4] Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) -(23) BroadcastExchange +(21) BroadcastExchange Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] Right keys [1]: [cs_item_sk#7] Join type: Inner Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) -(25) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] -(26) ReusedExchange [Reuses operator id: 38] +(24) ReusedExchange [Reuses operator id: 34] Output [1]: [d_date_sk#18] -(27) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [1]: [cs_ext_discount_amt#2] Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#18] -(29) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 6] Input [1]: [cs_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] Aggregate Attributes [1]: [sum#19] Results [1]: [sum#20] -(30) RowToColumnar -Input [1]: [sum#20] - -(31) CometColumnarExchange -Input [1]: [sum#20] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(32) ColumnarToRow [codegen id : 7] +(28) Exchange Input [1]: [sum#20] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(33) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 7] Input [1]: [sum#20] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] @@ -191,11 +175,11 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#21,17,2) AS ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) (unknown) Scan parquet spark_catalog.default.date_dim @@ -205,18 +189,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [2]: [d_date_sk#18, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) -(36) CometProject +(32) CometProject Input [2]: [d_date_sk#18, d_date#23] Arguments: [d_date_sk#18], [d_date_sk#18] -(37) ColumnarToRow [codegen id : 1] +(33) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#18] -(38) BroadcastExchange +(34) BroadcastExchange Input [1]: [d_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index a0ed5aafdb..146a33fdd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -1,56 +1,52 @@ WholeStageCodegen (7) HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [cs_ext_discount_amt] [sum,sum] - Project [cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_discount_amt,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] - Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow + InputAdapter + Exchange #1 + WholeStageCodegen (6) + HashAggregate [cs_ext_discount_amt] [sum,sum] + Project [cs_ext_discount_amt] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_discount_amt,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] + Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_ext_discount_amt] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Filter [(1.3 * avg(cs_ext_discount_amt))] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] InputAdapter - CometFilter [cs_item_sk,cs_ext_discount_amt] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + Exchange [cs_item_sk] #5 + WholeStageCodegen (3) + HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] + Project [cs_item_sk,cs_ext_discount_amt] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #5 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] - Project [cs_item_sk,cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index aa365f08ea..d7dda8078a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -1,75 +1,67 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * HashAggregate (70) - +- * ColumnarToRow (69) - +- CometColumnarExchange (68) - +- RowToColumnar (67) - +- * HashAggregate (66) - +- Union (65) - :- * HashAggregate (30) - : +- * ColumnarToRow (29) - : +- CometColumnarExchange (28) - : +- RowToColumnar (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - :- * HashAggregate (47) - : +- * ColumnarToRow (46) - : +- CometColumnarExchange (45) - : +- RowToColumnar (44) - : +- * HashAggregate (43) - : +- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (36) - : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : :- * ColumnarToRow (33) - : : : : +- CometFilter (32) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : : +- ReusedExchange (34) - : : +- ReusedExchange (37) - : +- ReusedExchange (40) - +- * HashAggregate (64) - +- * ColumnarToRow (63) - +- CometColumnarExchange (62) - +- RowToColumnar (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * ColumnarToRow (50) - : : : +- CometFilter (49) - : : : +- CometScan parquet spark_catalog.default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) +TakeOrderedAndProject (63) ++- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * ColumnarToRow (31) + : : : : +- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * ColumnarToRow (46) + : : : +- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (unknown) Scan parquet spark_catalog.default.store_sales @@ -87,7 +79,7 @@ Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 76] +(4) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -195,17 +187,11 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#13] Results [2]: [i_manufact_id#10, sum#14] -(27) RowToColumnar +(27) Exchange Input [2]: [i_manufact_id#10, sum#14] +Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) CometColumnarExchange -Input [2]: [i_manufact_id#10, sum#14] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) ColumnarToRow [codegen id : 6] -Input [2]: [i_manufact_id#10, sum#14] - -(30) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 6] Input [2]: [i_manufact_id#10, sum#14] Keys [1]: [i_manufact_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -220,70 +206,64 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(32) CometFilter +(30) CometFilter Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(33) ColumnarToRow [codegen id : 11] +(31) ColumnarToRow [codegen id : 11] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -(34) ReusedExchange [Reuses operator id: 76] +(32) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#22] -(35) BroadcastHashJoin [codegen id : 11] +(33) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#20] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(36) Project [codegen id : 11] +(34) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -(37) ReusedExchange [Reuses operator id: 11] +(35) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#23] -(38) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#17] Right keys [1]: [ca_address_sk#23] Join type: Inner Join condition: None -(39) Project [codegen id : 11] +(37) Project [codegen id : 11] Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] -(40) ReusedExchange [Reuses operator id: 23] +(38) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#24, i_manufact_id#25] -(41) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#18] Right keys [1]: [i_item_sk#24] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(40) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#19, i_manufact_id#25] Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_manufact_id#25] -(43) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#19, i_manufact_id#25] Keys [1]: [i_manufact_id#25] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] Aggregate Attributes [1]: [sum#26] Results [2]: [i_manufact_id#25, sum#27] -(44) RowToColumnar -Input [2]: [i_manufact_id#25, sum#27] - -(45) CometColumnarExchange -Input [2]: [i_manufact_id#25, sum#27] -Arguments: hashpartitioning(i_manufact_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(46) ColumnarToRow [codegen id : 12] +(42) Exchange Input [2]: [i_manufact_id#25, sum#27] +Arguments: hashpartitioning(i_manufact_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(47) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 12] Input [2]: [i_manufact_id#25, sum#27] Keys [1]: [i_manufact_id#25] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] @@ -298,114 +278,102 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) CometFilter +(45) CometFilter Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) -(50) ColumnarToRow [codegen id : 17] +(46) ColumnarToRow [codegen id : 17] Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -(51) ReusedExchange [Reuses operator id: 76] +(47) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#35] -(52) BroadcastHashJoin [codegen id : 17] +(48) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_sold_date_sk#33] Right keys [1]: [d_date_sk#35] Join type: Inner Join condition: None -(53) Project [codegen id : 17] +(49) Project [codegen id : 17] Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] -(54) ReusedExchange [Reuses operator id: 11] +(50) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#36] -(55) BroadcastHashJoin [codegen id : 17] +(51) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_bill_addr_sk#31] Right keys [1]: [ca_address_sk#36] Join type: Inner Join condition: None -(56) Project [codegen id : 17] +(52) Project [codegen id : 17] Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -(57) ReusedExchange [Reuses operator id: 23] +(53) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#37, i_manufact_id#38] -(58) BroadcastHashJoin [codegen id : 17] +(54) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_item_sk#30] Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(55) Project [codegen id : 17] Output [2]: [ws_ext_sales_price#32, i_manufact_id#38] Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_manufact_id#38] -(60) HashAggregate [codegen id : 17] +(56) HashAggregate [codegen id : 17] Input [2]: [ws_ext_sales_price#32, i_manufact_id#38] Keys [1]: [i_manufact_id#38] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] Aggregate Attributes [1]: [sum#39] Results [2]: [i_manufact_id#38, sum#40] -(61) RowToColumnar +(57) Exchange Input [2]: [i_manufact_id#38, sum#40] +Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(62) CometColumnarExchange -Input [2]: [i_manufact_id#38, sum#40] -Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(63) ColumnarToRow [codegen id : 18] -Input [2]: [i_manufact_id#38, sum#40] - -(64) HashAggregate [codegen id : 18] +(58) HashAggregate [codegen id : 18] Input [2]: [i_manufact_id#38, sum#40] Keys [1]: [i_manufact_id#38] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] Results [2]: [i_manufact_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] -(65) Union +(59) Union -(66) HashAggregate [codegen id : 19] +(60) HashAggregate [codegen id : 19] Input [2]: [i_manufact_id#10, total_sales#16] Keys [1]: [i_manufact_id#10] Functions [1]: [partial_sum(total_sales#16)] Aggregate Attributes [2]: [sum#43, isEmpty#44] Results [3]: [i_manufact_id#10, sum#45, isEmpty#46] -(67) RowToColumnar -Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] - -(68) CometColumnarExchange -Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(69) ColumnarToRow [codegen id : 20] +(61) Exchange Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] +Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(70) HashAggregate [codegen id : 20] +(62) HashAggregate [codegen id : 20] Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] Keys [1]: [i_manufact_id#10] Functions [1]: [sum(total_sales#16)] Aggregate Attributes [1]: [sum(total_sales#16)#47] Results [2]: [i_manufact_id#10, sum(total_sales#16)#47 AS total_sales#48] -(71) TakeOrderedAndProject +(63) TakeOrderedAndProject Input [2]: [i_manufact_id#10, total_sales#48] Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#10, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (68) ++- * ColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) (unknown) Scan parquet spark_catalog.default.date_dim @@ -415,23 +383,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(65) CometFilter Input [3]: [d_date_sk#6, d_year#49, d_moy#50] Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 5)) AND isnotnull(d_date_sk#6)) -(74) CometProject +(66) CometProject Input [3]: [d_date_sk#6, d_year#49, d_moy#50] Arguments: [d_date_sk#6], [d_date_sk#6] -(75) ColumnarToRow [codegen id : 1] +(67) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(76) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 5735fbbc06..4ab82379ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -1,113 +1,105 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (20) HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #1 - RowToColumnar - WholeStageCodegen (19) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) + InputAdapter + Exchange [i_manufact_id] #1 + WholeStageCodegen (19) + HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #2 + WholeStageCodegen (5) + HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_manufact_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (12) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #7 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [ca_address_sk] #4 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_manufact_id,i_manufact_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 - WholeStageCodegen (18) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #8 - RowToColumnar - WholeStageCodegen (17) - HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 + CometProject [i_manufact_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] + WholeStageCodegen (12) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #7 + WholeStageCodegen (11) + HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_manufact_id] #5 + WholeStageCodegen (18) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #8 + WholeStageCodegen (17) + HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt index 9e9b7dbf88..64f2b3c312 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt @@ -1,40 +1,36 @@ == Physical Plan == -* ColumnarToRow (36) -+- CometSort (35) - +- CometColumnarExchange (34) - +- RowToColumnar (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (26) - : +- * HashAggregate (25) - : +- * ColumnarToRow (24) - : +- CometColumnarExchange (23) - : +- RowToColumnar (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * ColumnarToRow (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.customer (27) +* Sort (32) ++- Exchange (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * Filter (24) + : +- * HashAggregate (23) + : +- Exchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (28) + +- * ColumnarToRow (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (unknown) Scan parquet spark_catalog.default.store_sales @@ -52,7 +48,7 @@ Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnu (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 41] +(4) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -136,24 +132,18 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#14] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) RowToColumnar +(22) Exchange Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(23) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) ColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(25) HashAggregate [codegen id : 6] +(23) HashAggregate [codegen id : 6] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(26) Filter [codegen id : 6] +(24) Filter [codegen id : 6] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) @@ -164,49 +154,43 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(28) CometFilter +(26) CometFilter Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Condition : isnotnull(c_customer_sk#18) -(29) ColumnarToRow [codegen id : 5] +(27) ColumnarToRow [codegen id : 5] Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(30) BroadcastExchange +(28) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(30) Project [codegen id : 6] Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(33) RowToColumnar -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] - -(34) CometColumnarExchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(35) CometSort +(31) Exchange Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST] +Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(36) ColumnarToRow [codegen id : 7] +(32) Sort [codegen id : 7] Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (41) -+- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (unknown) Scan parquet spark_catalog.default.date_dim @@ -216,18 +200,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(38) CometFilter +(34) CometFilter Input [3]: [d_date_sk#7, d_year#23, d_dom#24] Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(39) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#23, d_dom#24] Arguments: [d_date_sk#7], [d_date_sk#7] -(40) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(41) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index 0b46527a9e..80405a784d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -1,60 +1,56 @@ WholeStageCodegen (7) - ColumnarToRow + Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - RowToColumnar - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 + Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + WholeStageCodegen (6) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 5205f8b0e9..de7514efce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -1,48 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * ColumnarToRow (42) - +- CometColumnarExchange (41) - +- RowToColumnar (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * Filter (25) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_demographics (33) (unknown) Scan parquet spark_catalog.default.customer @@ -69,7 +67,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) ReusedExchange [Reuses operator id: 49] +(6) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#9] (7) BroadcastHashJoin [codegen id : 2] @@ -102,7 +100,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(13) ReusedExchange [Reuses operator id: 49] +(13) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#13] (14) BroadcastHashJoin [codegen id : 4] @@ -135,7 +133,7 @@ ReadSchema: struct (19) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -(20) ReusedExchange [Reuses operator id: 49] +(20) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 6] @@ -229,35 +227,29 @@ Functions [10]: [partial_count(1), partial_min(cd_dep_count#23), partial_max(cd_ Aggregate Attributes [13]: [count#26, min#27, max#28, sum#29, count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38] Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] -(40) RowToColumnar +(40) Exchange Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] +Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) CometColumnarExchange -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) ColumnarToRow [codegen id : 10] -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] - -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 10] Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Functions [10]: [count(1), min(cd_dep_count#23), max(cd_dep_count#23), avg(cd_dep_count#23), min(cd_dep_employed_count#24), max(cd_dep_employed_count#24), avg(cd_dep_employed_count#24), min(cd_dep_college_count#25), max(cd_dep_college_count#25), avg(cd_dep_college_count#25)] Aggregate Attributes [10]: [count(1)#52, min(cd_dep_count#23)#53, max(cd_dep_count#23)#54, avg(cd_dep_count#23)#55, min(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, avg(cd_dep_employed_count#24)#58, min(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, avg(cd_dep_college_count#25)#61] Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, count(1)#52 AS cnt1#62, min(cd_dep_count#23)#53 AS min(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, avg(cd_dep_count#23)#55 AS avg(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, min(cd_dep_employed_count#24)#56 AS min(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, avg(cd_dep_employed_count#24)#58 AS avg(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, min(cd_dep_college_count#25)#59 AS min(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, avg(cd_dep_college_count#25)#61 AS avg(cd_dep_college_count)#73, cd_dep_count#23] -(44) TakeOrderedAndProject +(42) TakeOrderedAndProject Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73, cd_dep_count#23] Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (unknown) Scan parquet spark_catalog.default.date_dim @@ -267,18 +259,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(44) CometFilter Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) -(47) CometProject +(45) CometProject Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] Arguments: [d_date_sk#9], [d_date_sk#9] -(48) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index 7dd83ba9b0..ea0ef274ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -1,76 +1,74 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] WholeStageCodegen (10) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt index 61f5a576fa..bb2a1b1a5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt @@ -1,36 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- Window (30) - +- * ColumnarToRow (29) - +- CometSort (28) - +- CometColumnarExchange (27) - +- RowToColumnar (26) - +- * HashAggregate (25) - +- * ColumnarToRow (24) - +- CometColumnarExchange (23) - +- RowToColumnar (22) - +- * HashAggregate (21) - +- * Expand (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store (13) +TakeOrderedAndProject (28) ++- * Project (27) + +- Window (26) + +- * Sort (25) + +- Exchange (24) + +- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * Expand (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometProject (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.store (13) (unknown) Scan parquet spark_catalog.default.store_sales @@ -48,7 +44,7 @@ Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 37] +(4) ReusedExchange [Reuses operator id: 33] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -132,57 +128,45 @@ Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(Unscale Aggregate Attributes [2]: [sum#16, sum#17] Results [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] -(22) RowToColumnar +(22) Exchange Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] +Arguments: hashpartitioning(i_category#13, i_class#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(23) CometColumnarExchange -Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] -Arguments: hashpartitioning(i_category#13, i_class#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) ColumnarToRow [codegen id : 5] -Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] - -(25) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 5] Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledValue(ss_ext_sales_price#3))#21] Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS _w0#24, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS _w1#25, CASE WHEN (cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint) = 0) THEN i_category#13 END AS _w2#26] -(26) RowToColumnar -Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] - -(27) CometColumnarExchange -Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -Arguments: hashpartitioning(_w1#25, _w2#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(28) CometSort +(24) Exchange Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -Arguments: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26], [_w1#25 ASC NULLS FIRST, _w2#26 ASC NULLS FIRST, _w0#24 ASC NULLS FIRST] +Arguments: hashpartitioning(_w1#25, _w2#26, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) ColumnarToRow [codegen id : 6] +(25) Sort [codegen id : 6] Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] +Arguments: [_w1#25 ASC NULLS FIRST, _w2#26 ASC NULLS FIRST, _w0#24 ASC NULLS FIRST], false, 0 -(30) Window +(26) Window Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] Arguments: [rank(_w0#24) windowspecdefinition(_w1#25, _w2#26, _w0#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#27], [_w1#25, _w2#26], [_w0#24 ASC NULLS FIRST] -(31) Project [codegen id : 7] +(27) Project [codegen id : 7] Output [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] Input [8]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26, rank_within_parent#27] -(32) TakeOrderedAndProject +(28) TakeOrderedAndProject Input [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] Arguments: 100, [lochierarchy#23 DESC NULLS LAST, CASE WHEN (lochierarchy#23 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#27 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) (unknown) Scan parquet spark_catalog.default.date_dim @@ -192,18 +176,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(30) CometFilter Input [2]: [d_date_sk#7, d_year#28] Condition : ((isnotnull(d_year#28) AND (d_year#28 = 2001)) AND isnotnull(d_date_sk#7)) -(35) CometProject +(31) CometProject Input [2]: [d_date_sk#7, d_year#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(32) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(37) BroadcastExchange +(33) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index f55a126f07..7eeb607c3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -4,52 +4,48 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (6) - ColumnarToRow + Sort [_w1,_w2,_w0] InputAdapter - CometSort [_w1,_w2,_w0] - CometColumnarExchange [_w1,_w2] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] - Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #3 + Exchange [_w1,_w2] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] + InputAdapter + Exchange [i_category,i_class,spark_grouping_id] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] + Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt index 06405034bc..d13ff264cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt @@ -1,31 +1,29 @@ == Physical Plan == -* ColumnarToRow (27) -+- CometTakeOrderedAndProject (26) - +- CometHashAggregate (25) - +- CometColumnarExchange (24) - +- RowToColumnar (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * ColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan parquet spark_catalog.default.catalog_sales (16) +TakeOrderedAndProject (25) ++- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildLeft (20) + :- BroadcastExchange (15) + : +- * Project (14) + : +- * BroadcastHashJoin Inner BuildRight (13) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (9) + : : +- * ColumnarToRow (8) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.inventory (5) + : +- ReusedExchange (12) + +- * ColumnarToRow (19) + +- CometProject (18) + +- CometFilter (17) + +- CometScan parquet spark_catalog.default.catalog_sales (16) (unknown) Scan parquet spark_catalog.default.item @@ -79,7 +77,7 @@ Join condition: None Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_item_sk#6, inv_date_sk#8] -(12) ReusedExchange [Reuses operator id: 32] +(12) ReusedExchange [Reuses operator id: 30] Output [1]: [d_date_sk#10] (13) BroadcastHashJoin [codegen id : 3] @@ -131,33 +129,29 @@ Functions: [] Aggregate Attributes: [] Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(23) RowToColumnar +(23) Exchange Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) CometColumnarExchange -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(25) CometHashAggregate +(24) HashAggregate [codegen id : 5] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] +Aggregate Attributes: [] +Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(26) CometTakeOrderedAndProject -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] - -(27) ColumnarToRow [codegen id : 5] +(25) TakeOrderedAndProject Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (32) -+- * ColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan parquet spark_catalog.default.date_dim (28) +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) (unknown) Scan parquet spark_catalog.default.date_dim @@ -167,18 +161,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] ReadSchema: struct -(29) CometFilter +(27) CometFilter Input [2]: [d_date_sk#10, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-01)) AND (d_date#13 <= 2000-04-01)) AND isnotnull(d_date_sk#10)) -(30) CometProject +(28) CometProject Input [2]: [d_date_sk#10, d_date#13] Arguments: [d_date_sk#10], [d_date_sk#10] -(31) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(32) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt index 3f84fa0598..65bb06348b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt @@ -1,46 +1,44 @@ -WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + WholeStageCodegen (5) + HashAggregate [i_item_id,i_item_desc,i_current_price] + InputAdapter + Exchange [i_item_id,i_item_desc,i_current_price] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,cs_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (3) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] InputAdapter - ReusedExchange [d_date_sk] #4 - ColumnarToRow - InputAdapter - CometProject [cs_item_sk] - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + ColumnarToRow + InputAdapter + CometProject [cs_item_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index 50b19fe4ae..87d960592d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -1,59 +1,51 @@ == Physical Plan == -* HashAggregate (55) -+- * ColumnarToRow (54) - +- CometColumnarExchange (53) - +- RowToColumnar (52) - +- * HashAggregate (51) - +- * Project (50) - +- * BroadcastHashJoin LeftSemi BuildRight (49) - :- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * ColumnarToRow (17) - : : +- CometHashAggregate (16) - : : +- CometColumnarExchange (15) - : : +- RowToColumnar (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer (7) - : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) - : +- CometHashAggregate (30) - : +- CometColumnarExchange (29) - : +- RowToColumnar (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * ColumnarToRow (20) - : : : +- CometFilter (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (48) - +- * ColumnarToRow (47) - +- CometHashAggregate (46) - +- CometColumnarExchange (45) - +- RowToColumnar (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * ColumnarToRow (36) - : : +- CometFilter (35) - : : +- CometScan parquet spark_catalog.default.web_sales (34) - : +- ReusedExchange (37) - +- ReusedExchange (40) +* HashAggregate (47) ++- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin LeftSemi BuildRight (43) + :- * BroadcastHashJoin LeftSemi BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer (7) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- BroadcastExchange (42) + +- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * ColumnarToRow (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.web_sales (30) + : +- ReusedExchange (33) + +- ReusedExchange (36) (unknown) Scan parquet spark_catalog.default.store_sales @@ -71,7 +63,7 @@ Condition : isnotnull(ss_customer_sk#1) (3) ColumnarToRow [codegen id : 3] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -(4) ReusedExchange [Reuses operator id: 60] +(4) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#4, d_date#5] (5) BroadcastHashJoin [codegen id : 3] @@ -119,20 +111,16 @@ Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#8, c_first_name#7, d_date#5] -(14) RowToColumnar +(14) Exchange Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometHashAggregate +(15) HashAggregate [codegen id : 12] Input [3]: [c_last_name#8, c_first_name#7, d_date#5] Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] Functions: [] - -(17) ColumnarToRow [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Aggregate Attributes: [] +Results [3]: [c_last_name#8, c_first_name#7, d_date#5] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] @@ -142,66 +130,62 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] Condition : isnotnull(cs_bill_customer_sk#9) -(20) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 6] Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] -(21) ReusedExchange [Reuses operator id: 60] +(19) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#12, d_date#13] -(22) BroadcastHashJoin [codegen id : 6] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(23) Project [codegen id : 6] +(21) Project [codegen id : 6] Output [2]: [cs_bill_customer_sk#9, d_date#13] Input [4]: [cs_bill_customer_sk#9, cs_sold_date_sk#10, d_date_sk#12, d_date#13] -(24) ReusedExchange [Reuses operator id: 10] +(22) ReusedExchange [Reuses operator id: 10] Output [3]: [c_customer_sk#14, c_first_name#15, c_last_name#16] -(25) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_bill_customer_sk#9] Right keys [1]: [c_customer_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [3]: [c_last_name#16, c_first_name#15, d_date#13] Input [5]: [cs_bill_customer_sk#9, d_date#13, c_customer_sk#14, c_first_name#15, c_last_name#16] -(27) HashAggregate [codegen id : 6] +(25) HashAggregate [codegen id : 6] Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(28) RowToColumnar +(26) Exchange Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(29) CometColumnarExchange -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) CometHashAggregate +(27) HashAggregate [codegen id : 7] Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(31) ColumnarToRow [codegen id : 7] -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] - -(32) BroadcastExchange +(28) BroadcastExchange Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 12] +(29) BroadcastHashJoin [codegen id : 12] Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#16, ), isnull(c_last_name#16), coalesce(c_first_name#15, ), isnull(c_first_name#15), coalesce(d_date#13, 1970-01-01), isnull(d_date#13)] Join type: LeftSemi @@ -215,93 +199,83 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#18), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] Condition : isnotnull(ws_bill_customer_sk#17) -(36) ColumnarToRow [codegen id : 10] +(32) ColumnarToRow [codegen id : 10] Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] -(37) ReusedExchange [Reuses operator id: 60] +(33) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#20, d_date#21] -(38) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#18] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(39) Project [codegen id : 10] +(35) Project [codegen id : 10] Output [2]: [ws_bill_customer_sk#17, d_date#21] Input [4]: [ws_bill_customer_sk#17, ws_sold_date_sk#18, d_date_sk#20, d_date#21] -(40) ReusedExchange [Reuses operator id: 10] +(36) ReusedExchange [Reuses operator id: 10] Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] -(41) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_bill_customer_sk#17] Right keys [1]: [c_customer_sk#22] Join type: Inner Join condition: None -(42) Project [codegen id : 10] +(38) Project [codegen id : 10] Output [3]: [c_last_name#24, c_first_name#23, d_date#21] Input [5]: [ws_bill_customer_sk#17, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] -(43) HashAggregate [codegen id : 10] +(39) HashAggregate [codegen id : 10] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(44) RowToColumnar -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(45) CometColumnarExchange +(40) Exchange Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(46) CometHashAggregate +(41) HashAggregate [codegen id : 11] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(47) ColumnarToRow [codegen id : 11] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(48) BroadcastExchange +(42) BroadcastExchange Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 12] +(43) BroadcastHashJoin [codegen id : 12] Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] Join type: LeftSemi Join condition: None -(50) Project [codegen id : 12] +(44) Project [codegen id : 12] Output: [] Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -(51) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#25] Results [1]: [count#26] -(52) RowToColumnar -Input [1]: [count#26] - -(53) CometColumnarExchange -Input [1]: [count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(54) ColumnarToRow [codegen id : 13] +(46) Exchange Input [1]: [count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(55) HashAggregate [codegen id : 13] +(47) HashAggregate [codegen id : 13] Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] @@ -311,11 +285,11 @@ Results [1]: [count(1)#27 AS count(1)#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (60) -+- * ColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.date_dim (56) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim @@ -325,23 +299,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter +(49) CometFilter Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) -(58) CometProject +(50) CometProject Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(59) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(60) BroadcastExchange +(52) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 +Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 34 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index c3e01e4f26..315afe6602 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -1,89 +1,81 @@ WholeStageCodegen (13) HashAggregate [count] [count(1),count(1),count] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (12) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - ColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + InputAdapter + Exchange #1 + WholeStageCodegen (12) + HashAggregate [count,count] + Project + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index 11adf85554..e10ff3340d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -1,54 +1,48 @@ == Physical Plan == -* ColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- RowToColumnar (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (25) - : +- * Filter (24) - : +- * HashAggregate (23) - : +- * ColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- RowToColumnar (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (45) - +- * Project (44) - +- * Filter (43) - +- * HashAggregate (42) - +- * ColumnarToRow (41) - +- CometColumnarExchange (40) - +- RowToColumnar (39) - +- * HashAggregate (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- CometFilter (27) - : : : +- CometScan parquet spark_catalog.default.inventory (26) - : : +- ReusedExchange (29) - : +- ReusedExchange (32) - +- ReusedExchange (35) +* Sort (44) ++- Exchange (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (23) + : +- * Filter (22) + : +- * HashAggregate (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.warehouse (10) + : +- ReusedExchange (16) + +- BroadcastExchange (41) + +- * Project (40) + +- * Filter (39) + +- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * ColumnarToRow (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.inventory (24) + : : +- ReusedExchange (27) + : +- ReusedExchange (30) + +- ReusedExchange (33) (unknown) Scan parquet spark_catalog.default.inventory @@ -122,7 +116,7 @@ Join condition: None Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -(16) ReusedExchange [Reuses operator id: 55] +(16) ReusedExchange [Reuses operator id: 49] Output [2]: [d_date_sk#9, d_moy#10] (17) BroadcastHashJoin [codegen id : 4] @@ -142,28 +136,22 @@ Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), par Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(20) RowToColumnar +(20) Exchange Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometColumnarExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) ColumnarToRow [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(23) HashAggregate [codegen id : 10] +(21) HashAggregate [codegen id : 10] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] -(24) Filter [codegen id : 10] +(22) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END -(25) Project [codegen id : 10] +(23) Project [codegen id : 10] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] @@ -175,116 +163,104 @@ PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(27) CometFilter +(25) CometFilter Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) -(28) ColumnarToRow [codegen id : 8] +(26) ColumnarToRow [codegen id : 8] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -(29) ReusedExchange [Reuses operator id: 7] +(27) ReusedExchange [Reuses operator id: 7] Output [1]: [i_item_sk#31] -(30) BroadcastHashJoin [codegen id : 8] +(28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_item_sk#26] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(31) Project [codegen id : 8] +(29) Project [codegen id : 8] Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -(32) ReusedExchange [Reuses operator id: 13] +(30) ReusedExchange [Reuses operator id: 13] Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] -(33) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_warehouse_sk#27] Right keys [1]: [w_warehouse_sk#32] Join type: Inner Join condition: None -(34) Project [codegen id : 8] +(32) Project [codegen id : 8] Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -(35) ReusedExchange [Reuses operator id: 60] +(33) ReusedExchange [Reuses operator id: 54] Output [2]: [d_date_sk#34, d_moy#35] -(36) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(35) Project [codegen id : 8] Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] -(38) HashAggregate [codegen id : 8] +(36) HashAggregate [codegen id : 8] Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -(39) RowToColumnar -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(40) CometColumnarExchange +(37) Exchange Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(41) ColumnarToRow [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(42) HashAggregate [codegen id : 9] +(38) HashAggregate [codegen id : 9] Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] -(43) Filter [codegen id : 9] +(39) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END -(44) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -(45) BroadcastExchange +(41) BroadcastExchange Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(46) BroadcastHashJoin [codegen id : 10] +(42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] Join type: Inner Join condition: None -(47) RowToColumnar -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] - -(48) CometColumnarExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(49) CometSort +(43) Exchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(50) ColumnarToRow [codegen id : 11] +(44) Sort [codegen id : 11] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] +Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) (unknown) Scan parquet spark_catalog.default.date_dim @@ -294,27 +270,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) -(53) CometProject +(47) CometProject Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] -(54) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#10] -(55) BroadcastExchange +(49) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 26 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (60) -+- * ColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.date_dim (56) +Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) (unknown) Scan parquet spark_catalog.default.date_dim @@ -324,18 +300,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter +(51) CometFilter Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) -(58) CometProject +(52) CometProject Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] -(59) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_moy#35] -(60) BroadcastExchange +(54) BroadcastExchange Input [2]: [d_date_sk#34, d_moy#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 0dc072692c..002266e76e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -1,87 +1,81 @@ WholeStageCodegen (11) - ColumnarToRow + Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - RowToColumnar - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 + Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - ColumnarToRow + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + ReusedExchange [d_date_sk,d_moy] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (9) + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + WholeStageCodegen (8) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [i_item_sk] #4 + InputAdapter + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + InputAdapter + ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index cf500fbd99..98e8bc4642 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -1,54 +1,48 @@ == Physical Plan == -* ColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- RowToColumnar (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (25) - : +- * Filter (24) - : +- * HashAggregate (23) - : +- * ColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- RowToColumnar (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (45) - +- * Project (44) - +- * Filter (43) - +- * HashAggregate (42) - +- * ColumnarToRow (41) - +- CometColumnarExchange (40) - +- RowToColumnar (39) - +- * HashAggregate (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- CometFilter (27) - : : : +- CometScan parquet spark_catalog.default.inventory (26) - : : +- ReusedExchange (29) - : +- ReusedExchange (32) - +- ReusedExchange (35) +* Sort (44) ++- Exchange (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (23) + : +- * Filter (22) + : +- * HashAggregate (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.warehouse (10) + : +- ReusedExchange (16) + +- BroadcastExchange (41) + +- * Project (40) + +- * Filter (39) + +- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * ColumnarToRow (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.inventory (24) + : : +- ReusedExchange (27) + : +- ReusedExchange (30) + +- ReusedExchange (33) (unknown) Scan parquet spark_catalog.default.inventory @@ -122,7 +116,7 @@ Join condition: None Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -(16) ReusedExchange [Reuses operator id: 55] +(16) ReusedExchange [Reuses operator id: 49] Output [2]: [d_date_sk#9, d_moy#10] (17) BroadcastHashJoin [codegen id : 4] @@ -142,28 +136,22 @@ Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), par Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(20) RowToColumnar +(20) Exchange Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometColumnarExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) ColumnarToRow [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(23) HashAggregate [codegen id : 10] +(21) HashAggregate [codegen id : 10] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] -(24) Filter [codegen id : 10] +(22) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] Condition : (CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END AND CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.5) END) -(25) Project [codegen id : 10] +(23) Project [codegen id : 10] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] @@ -175,116 +163,104 @@ PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(27) CometFilter +(25) CometFilter Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) -(28) ColumnarToRow [codegen id : 8] +(26) ColumnarToRow [codegen id : 8] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -(29) ReusedExchange [Reuses operator id: 7] +(27) ReusedExchange [Reuses operator id: 7] Output [1]: [i_item_sk#31] -(30) BroadcastHashJoin [codegen id : 8] +(28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_item_sk#26] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(31) Project [codegen id : 8] +(29) Project [codegen id : 8] Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -(32) ReusedExchange [Reuses operator id: 13] +(30) ReusedExchange [Reuses operator id: 13] Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] -(33) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_warehouse_sk#27] Right keys [1]: [w_warehouse_sk#32] Join type: Inner Join condition: None -(34) Project [codegen id : 8] +(32) Project [codegen id : 8] Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -(35) ReusedExchange [Reuses operator id: 60] +(33) ReusedExchange [Reuses operator id: 54] Output [2]: [d_date_sk#34, d_moy#35] -(36) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [inv_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(35) Project [codegen id : 8] Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] -(38) HashAggregate [codegen id : 8] +(36) HashAggregate [codegen id : 8] Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -(39) RowToColumnar -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(40) CometColumnarExchange +(37) Exchange Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(41) ColumnarToRow [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(42) HashAggregate [codegen id : 9] +(38) HashAggregate [codegen id : 9] Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] -(43) Filter [codegen id : 9] +(39) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END -(44) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -(45) BroadcastExchange +(41) BroadcastExchange Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(46) BroadcastHashJoin [codegen id : 10] +(42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] Join type: Inner Join condition: None -(47) RowToColumnar -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] - -(48) CometColumnarExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(49) CometSort +(43) Exchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(50) ColumnarToRow [codegen id : 11] +(44) Sort [codegen id : 11] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] +Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) (unknown) Scan parquet spark_catalog.default.date_dim @@ -294,27 +270,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) -(53) CometProject +(47) CometProject Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] -(54) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#10] -(55) BroadcastExchange +(49) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 26 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (60) -+- * ColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.date_dim (56) +Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) (unknown) Scan parquet spark_catalog.default.date_dim @@ -324,18 +300,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter +(51) CometFilter Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) -(58) CometProject +(52) CometProject Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] -(59) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_moy#35] -(60) BroadcastExchange +(54) BroadcastExchange Input [2]: [d_date_sk#34, d_moy#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 0dc072692c..002266e76e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -1,87 +1,81 @@ WholeStageCodegen (11) - ColumnarToRow + Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - RowToColumnar - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 + Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - ColumnarToRow + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + ReusedExchange [d_date_sk,d_moy] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (9) + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + WholeStageCodegen (8) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [i_item_sk] #4 + InputAdapter + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + InputAdapter + ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index 1866b9e21a..9b2bb49916 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -1,124 +1,112 @@ == Physical Plan == -TakeOrderedAndProject (120) -+- * Project (119) - +- * BroadcastHashJoin Inner BuildRight (118) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- * Project (78) - : : +- * BroadcastHashJoin Inner BuildRight (77) - : : :- * Project (58) - : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : :- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Filter (18) - : : : : : +- * HashAggregate (17) - : : : : : +- * ColumnarToRow (16) - : : : : : +- CometColumnarExchange (15) - : : : : : +- RowToColumnar (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (10) - : : : : +- BroadcastExchange (36) - : : : : +- * HashAggregate (35) - : : : : +- * ColumnarToRow (34) - : : : : +- CometColumnarExchange (33) - : : : : +- RowToColumnar (32) - : : : : +- * HashAggregate (31) - : : : : +- * Project (30) - : : : : +- * BroadcastHashJoin Inner BuildRight (29) - : : : : :- * Project (27) - : : : : : +- * BroadcastHashJoin Inner BuildRight (26) - : : : : : :- * ColumnarToRow (21) - : : : : : : +- CometFilter (20) - : : : : : : +- CometScan parquet spark_catalog.default.customer (19) - : : : : : +- BroadcastExchange (25) - : : : : : +- * ColumnarToRow (24) - : : : : : +- CometFilter (23) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (22) - : : : : +- ReusedExchange (28) - : : : +- BroadcastExchange (56) - : : : +- * Filter (55) - : : : +- * HashAggregate (54) - : : : +- * ColumnarToRow (53) - : : : +- CometColumnarExchange (52) - : : : +- RowToColumnar (51) - : : : +- * HashAggregate (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * Project (46) - : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : :- * ColumnarToRow (40) - : : : : : +- CometFilter (39) - : : : : : +- CometScan parquet spark_catalog.default.customer (38) - : : : : +- BroadcastExchange (44) - : : : : +- * ColumnarToRow (43) - : : : : +- CometFilter (42) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) - : : : +- ReusedExchange (47) - : : +- BroadcastExchange (76) - : : +- * HashAggregate (75) - : : +- * ColumnarToRow (74) - : : +- CometColumnarExchange (73) - : : +- RowToColumnar (72) - : : +- * HashAggregate (71) - : : +- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * Project (67) - : : : +- * BroadcastHashJoin Inner BuildRight (66) - : : : :- * ColumnarToRow (61) - : : : : +- CometFilter (60) - : : : : +- CometScan parquet spark_catalog.default.customer (59) - : : : +- BroadcastExchange (65) - : : : +- * ColumnarToRow (64) - : : : +- CometFilter (63) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (62) - : : +- ReusedExchange (68) - : +- BroadcastExchange (97) - : +- * Filter (96) - : +- * HashAggregate (95) - : +- * ColumnarToRow (94) - : +- CometColumnarExchange (93) - : +- RowToColumnar (92) - : +- * HashAggregate (91) - : +- * Project (90) - : +- * BroadcastHashJoin Inner BuildRight (89) - : :- * Project (87) - : : +- * BroadcastHashJoin Inner BuildRight (86) - : : :- * ColumnarToRow (81) - : : : +- CometFilter (80) - : : : +- CometScan parquet spark_catalog.default.customer (79) - : : +- BroadcastExchange (85) - : : +- * ColumnarToRow (84) - : : +- CometFilter (83) - : : +- CometScan parquet spark_catalog.default.web_sales (82) - : +- ReusedExchange (88) - +- BroadcastExchange (117) - +- * HashAggregate (116) - +- * ColumnarToRow (115) - +- CometColumnarExchange (114) - +- RowToColumnar (113) - +- * HashAggregate (112) - +- * Project (111) - +- * BroadcastHashJoin Inner BuildRight (110) - :- * Project (108) - : +- * BroadcastHashJoin Inner BuildRight (107) - : :- * ColumnarToRow (102) - : : +- CometFilter (101) - : : +- CometScan parquet spark_catalog.default.customer (100) - : +- BroadcastExchange (106) - : +- * ColumnarToRow (105) - : +- CometFilter (104) - : +- CometScan parquet spark_catalog.default.web_sales (103) - +- ReusedExchange (109) +TakeOrderedAndProject (108) ++- * Project (107) + +- * BroadcastHashJoin Inner BuildRight (106) + :- * Project (89) + : +- * BroadcastHashJoin Inner BuildRight (88) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * Project (52) + : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : :- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Filter (16) + : : : : : +- * HashAggregate (15) + : : : : : +- Exchange (14) + : : : : : +- * HashAggregate (13) + : : : : : +- * Project (12) + : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : : +- ReusedExchange (10) + : : : : +- BroadcastExchange (32) + : : : : +- * HashAggregate (31) + : : : : +- Exchange (30) + : : : : +- * HashAggregate (29) + : : : : +- * Project (28) + : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : :- * Project (25) + : : : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : : : :- * ColumnarToRow (19) + : : : : : : +- CometFilter (18) + : : : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : : : +- BroadcastExchange (23) + : : : : : +- * ColumnarToRow (22) + : : : : : +- CometFilter (21) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : : : +- ReusedExchange (26) + : : : +- BroadcastExchange (50) + : : : +- * Filter (49) + : : : +- * HashAggregate (48) + : : : +- Exchange (47) + : : : +- * HashAggregate (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (42) + : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : :- * ColumnarToRow (36) + : : : : : +- CometFilter (35) + : : : : : +- CometScan parquet spark_catalog.default.customer (34) + : : : : +- BroadcastExchange (40) + : : : : +- * ColumnarToRow (39) + : : : : +- CometFilter (38) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (37) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (68) + : : +- * HashAggregate (67) + : : +- Exchange (66) + : : +- * HashAggregate (65) + : : +- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (61) + : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : :- * ColumnarToRow (55) + : : : : +- CometFilter (54) + : : : : +- CometScan parquet spark_catalog.default.customer (53) + : : : +- BroadcastExchange (59) + : : : +- * ColumnarToRow (58) + : : : +- CometFilter (57) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (56) + : : +- ReusedExchange (62) + : +- BroadcastExchange (87) + : +- * Filter (86) + : +- * HashAggregate (85) + : +- Exchange (84) + : +- * HashAggregate (83) + : +- * Project (82) + : +- * BroadcastHashJoin Inner BuildRight (81) + : :- * Project (79) + : : +- * BroadcastHashJoin Inner BuildRight (78) + : : :- * ColumnarToRow (73) + : : : +- CometFilter (72) + : : : +- CometScan parquet spark_catalog.default.customer (71) + : : +- BroadcastExchange (77) + : : +- * ColumnarToRow (76) + : : +- CometFilter (75) + : : +- CometScan parquet spark_catalog.default.web_sales (74) + : +- ReusedExchange (80) + +- BroadcastExchange (105) + +- * HashAggregate (104) + +- Exchange (103) + +- * HashAggregate (102) + +- * Project (101) + +- * BroadcastHashJoin Inner BuildRight (100) + :- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * ColumnarToRow (92) + : : +- CometFilter (91) + : : +- CometScan parquet spark_catalog.default.customer (90) + : +- BroadcastExchange (96) + : +- * ColumnarToRow (95) + : +- CometFilter (94) + : +- CometScan parquet spark_catalog.default.web_sales (93) + +- ReusedExchange (99) (unknown) Scan parquet spark_catalog.default.customer @@ -164,7 +152,7 @@ Join condition: None Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -(10) ReusedExchange [Reuses operator id: 124] +(10) ReusedExchange [Reuses operator id: 112] Output [2]: [d_date_sk#16, d_year#17] (11) BroadcastHashJoin [codegen id : 3] @@ -184,24 +172,18 @@ Functions [1]: [partial_sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] -(14) RowToColumnar +(14) Exchange Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 24] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] - -(17) HashAggregate [codegen id : 24] +(15) HashAggregate [codegen id : 24] Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] Functions [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))] Aggregate Attributes [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22] Results [2]: [c_customer_id#2 AS customer_id#23, sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22 AS year_total#24] -(18) Filter [codegen id : 24] +(16) Filter [codegen id : 24] Input [2]: [customer_id#23, year_total#24] Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.000000)) @@ -212,11 +194,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(20) CometFilter +(18) CometFilter Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_customer_id#26)) -(21) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 6] Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] (unknown) Scan parquet spark_catalog.default.store_sales @@ -227,69 +209,63 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter +(21) CometFilter Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] Condition : isnotnull(ss_customer_sk#33) -(24) ColumnarToRow [codegen id : 4] +(22) ColumnarToRow [codegen id : 4] Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] -(25) BroadcastExchange +(23) BroadcastExchange Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [c_customer_sk#25] Right keys [1]: [ss_customer_sk#33] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(25) Project [codegen id : 6] Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] Input [14]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] -(28) ReusedExchange [Reuses operator id: 128] +(26) ReusedExchange [Reuses operator id: 116] Output [2]: [d_date_sk#40, d_year#41] -(29) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#38] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(30) Project [codegen id : 6] +(28) Project [codegen id : 6] Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] Input [14]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] -(31) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] Functions [1]: [partial_sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] Aggregate Attributes [2]: [sum#42, isEmpty#43] Results [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] -(32) RowToColumnar -Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] - -(33) CometColumnarExchange +(30) Exchange Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] -Arguments: hashpartitioning(c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(34) ColumnarToRow [codegen id : 7] -Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] - -(35) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 7] Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] Functions [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] Aggregate Attributes [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22] Results [8]: [c_customer_id#26 AS customer_id#46, c_first_name#27 AS customer_first_name#47, c_last_name#28 AS customer_last_name#48, c_preferred_cust_flag#29 AS customer_preferred_cust_flag#49, c_birth_country#30 AS customer_birth_country#50, c_login#31 AS customer_login#51, c_email_address#32 AS customer_email_address#52, sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22 AS year_total#53] -(36) BroadcastExchange +(32) BroadcastExchange Input [8]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 24] +(33) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#46] Join type: Inner @@ -302,11 +278,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(39) CometFilter +(35) CometFilter Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] Condition : (isnotnull(c_customer_sk#54) AND isnotnull(c_customer_id#55)) -(40) ColumnarToRow [codegen id : 10] +(36) ColumnarToRow [codegen id : 10] Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -317,79 +293,73 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#67), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(42) CometFilter +(38) CometFilter Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] Condition : isnotnull(cs_bill_customer_sk#62) -(43) ColumnarToRow [codegen id : 8] +(39) ColumnarToRow [codegen id : 8] Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -(44) BroadcastExchange +(40) BroadcastExchange Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 10] +(41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#54] Right keys [1]: [cs_bill_customer_sk#62] Join type: Inner Join condition: None -(46) Project [codegen id : 10] +(42) Project [codegen id : 10] Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] Input [14]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -(47) ReusedExchange [Reuses operator id: 124] +(43) ReusedExchange [Reuses operator id: 112] Output [2]: [d_date_sk#69, d_year#70] -(48) BroadcastHashJoin [codegen id : 10] +(44) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#67] Right keys [1]: [d_date_sk#69] Join type: Inner Join condition: None -(49) Project [codegen id : 10] +(45) Project [codegen id : 10] Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] Input [14]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67, d_date_sk#69, d_year#70] -(50) HashAggregate [codegen id : 10] +(46) HashAggregate [codegen id : 10] Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] Functions [1]: [partial_sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] Aggregate Attributes [2]: [sum#71, isEmpty#72] Results [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] -(51) RowToColumnar -Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] - -(52) CometColumnarExchange -Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) ColumnarToRow [codegen id : 11] +(47) Exchange Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] +Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(54) HashAggregate [codegen id : 11] +(48) HashAggregate [codegen id : 11] Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] Functions [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] Aggregate Attributes [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75] Results [2]: [c_customer_id#55 AS customer_id#76, sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75 AS year_total#77] -(55) Filter [codegen id : 11] +(49) Filter [codegen id : 11] Input [2]: [customer_id#76, year_total#77] Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.000000)) -(56) BroadcastExchange +(50) BroadcastExchange Input [2]: [customer_id#76, year_total#77] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 24] +(51) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#76] Join type: Inner Join condition: None -(58) Project [codegen id : 24] +(52) Project [codegen id : 24] Output [11]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77] Input [12]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, customer_id#76, year_total#77] @@ -400,11 +370,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(60) CometFilter +(54) CometFilter Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] Condition : (isnotnull(c_customer_sk#78) AND isnotnull(c_customer_id#79)) -(61) ColumnarToRow [codegen id : 14] +(55) ColumnarToRow [codegen id : 14] Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -415,75 +385,69 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(63) CometFilter +(57) CometFilter Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] Condition : isnotnull(cs_bill_customer_sk#86) -(64) ColumnarToRow [codegen id : 12] +(58) ColumnarToRow [codegen id : 12] Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] -(65) BroadcastExchange +(59) BroadcastExchange Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(66) BroadcastHashJoin [codegen id : 14] +(60) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_customer_sk#78] Right keys [1]: [cs_bill_customer_sk#86] Join type: Inner Join condition: None -(67) Project [codegen id : 14] +(61) Project [codegen id : 14] Output [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] Input [14]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] -(68) ReusedExchange [Reuses operator id: 128] +(62) ReusedExchange [Reuses operator id: 116] Output [2]: [d_date_sk#93, d_year#94] -(69) BroadcastHashJoin [codegen id : 14] +(63) BroadcastHashJoin [codegen id : 14] Left keys [1]: [cs_sold_date_sk#91] Right keys [1]: [d_date_sk#93] Join type: Inner Join condition: None -(70) Project [codegen id : 14] +(64) Project [codegen id : 14] Output [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] Input [14]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91, d_date_sk#93, d_year#94] -(71) HashAggregate [codegen id : 14] +(65) HashAggregate [codegen id : 14] Input [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94] Functions [1]: [partial_sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))] Aggregate Attributes [2]: [sum#95, isEmpty#96] Results [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] -(72) RowToColumnar +(66) Exchange Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] +Arguments: hashpartitioning(c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(73) CometColumnarExchange -Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] -Arguments: hashpartitioning(c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) ColumnarToRow [codegen id : 15] -Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] - -(75) HashAggregate [codegen id : 15] +(67) HashAggregate [codegen id : 15] Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94] Functions [1]: [sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))] Aggregate Attributes [1]: [sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))#75] Results [2]: [c_customer_id#79 AS customer_id#99, sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))#75 AS year_total#100] -(76) BroadcastExchange +(68) BroadcastExchange Input [2]: [customer_id#99, year_total#100] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 24] +(69) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#99] Join type: Inner Join condition: (CASE WHEN (year_total#77 > 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END) -(78) Project [codegen id : 24] +(70) Project [codegen id : 24] Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100] Input [13]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77, customer_id#99, year_total#100] @@ -494,11 +458,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(80) CometFilter +(72) CometFilter Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] Condition : (isnotnull(c_customer_sk#101) AND isnotnull(c_customer_id#102)) -(81) ColumnarToRow [codegen id : 18] +(73) ColumnarToRow [codegen id : 18] Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] (unknown) Scan parquet spark_catalog.default.web_sales @@ -509,79 +473,73 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#114), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(83) CometFilter +(75) CometFilter Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] Condition : isnotnull(ws_bill_customer_sk#109) -(84) ColumnarToRow [codegen id : 16] +(76) ColumnarToRow [codegen id : 16] Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] -(85) BroadcastExchange +(77) BroadcastExchange Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(86) BroadcastHashJoin [codegen id : 18] +(78) BroadcastHashJoin [codegen id : 18] Left keys [1]: [c_customer_sk#101] Right keys [1]: [ws_bill_customer_sk#109] Join type: Inner Join condition: None -(87) Project [codegen id : 18] +(79) Project [codegen id : 18] Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] Input [14]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] -(88) ReusedExchange [Reuses operator id: 124] +(80) ReusedExchange [Reuses operator id: 112] Output [2]: [d_date_sk#116, d_year#117] -(89) BroadcastHashJoin [codegen id : 18] +(81) BroadcastHashJoin [codegen id : 18] Left keys [1]: [ws_sold_date_sk#114] Right keys [1]: [d_date_sk#116] Join type: Inner Join condition: None -(90) Project [codegen id : 18] +(82) Project [codegen id : 18] Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] Input [14]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114, d_date_sk#116, d_year#117] -(91) HashAggregate [codegen id : 18] +(83) HashAggregate [codegen id : 18] Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] Functions [1]: [partial_sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] Aggregate Attributes [2]: [sum#118, isEmpty#119] Results [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] -(92) RowToColumnar -Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] - -(93) CometColumnarExchange +(84) Exchange Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] -Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(94) ColumnarToRow [codegen id : 19] -Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] - -(95) HashAggregate [codegen id : 19] +(85) HashAggregate [codegen id : 19] Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] Functions [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] Aggregate Attributes [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122] Results [2]: [c_customer_id#102 AS customer_id#123, sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122 AS year_total#124] -(96) Filter [codegen id : 19] +(86) Filter [codegen id : 19] Input [2]: [customer_id#123, year_total#124] Condition : (isnotnull(year_total#124) AND (year_total#124 > 0.000000)) -(97) BroadcastExchange +(87) BroadcastExchange Input [2]: [customer_id#123, year_total#124] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] -(98) BroadcastHashJoin [codegen id : 24] +(88) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#123] Join type: Inner Join condition: None -(99) Project [codegen id : 24] +(89) Project [codegen id : 24] Output [11]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124] Input [12]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, customer_id#123, year_total#124] @@ -592,11 +550,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(101) CometFilter +(91) CometFilter Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] Condition : (isnotnull(c_customer_sk#125) AND isnotnull(c_customer_id#126)) -(102) ColumnarToRow [codegen id : 22] +(92) ColumnarToRow [codegen id : 22] Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] (unknown) Scan parquet spark_catalog.default.web_sales @@ -607,89 +565,83 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#138), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(104) CometFilter +(94) CometFilter Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] Condition : isnotnull(ws_bill_customer_sk#133) -(105) ColumnarToRow [codegen id : 20] +(95) ColumnarToRow [codegen id : 20] Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] -(106) BroadcastExchange +(96) BroadcastExchange Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -(107) BroadcastHashJoin [codegen id : 22] +(97) BroadcastHashJoin [codegen id : 22] Left keys [1]: [c_customer_sk#125] Right keys [1]: [ws_bill_customer_sk#133] Join type: Inner Join condition: None -(108) Project [codegen id : 22] +(98) Project [codegen id : 22] Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] Input [14]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] -(109) ReusedExchange [Reuses operator id: 128] +(99) ReusedExchange [Reuses operator id: 116] Output [2]: [d_date_sk#140, d_year#141] -(110) BroadcastHashJoin [codegen id : 22] +(100) BroadcastHashJoin [codegen id : 22] Left keys [1]: [ws_sold_date_sk#138] Right keys [1]: [d_date_sk#140] Join type: Inner Join condition: None -(111) Project [codegen id : 22] +(101) Project [codegen id : 22] Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] Input [14]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138, d_date_sk#140, d_year#141] -(112) HashAggregate [codegen id : 22] +(102) HashAggregate [codegen id : 22] Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] Functions [1]: [partial_sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] Aggregate Attributes [2]: [sum#142, isEmpty#143] Results [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] -(113) RowToColumnar -Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] - -(114) CometColumnarExchange -Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] -Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(115) ColumnarToRow [codegen id : 23] +(103) Exchange Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] +Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(116) HashAggregate [codegen id : 23] +(104) HashAggregate [codegen id : 23] Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] Functions [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] Aggregate Attributes [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122] Results [2]: [c_customer_id#126 AS customer_id#146, sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122 AS year_total#147] -(117) BroadcastExchange +(105) BroadcastExchange Input [2]: [customer_id#146, year_total#147] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] -(118) BroadcastHashJoin [codegen id : 24] +(106) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#146] Join type: Inner Join condition: (CASE WHEN (year_total#77 > 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#124 > 0.000000) THEN (year_total#147 / year_total#124) END) -(119) Project [codegen id : 24] +(107) Project [codegen id : 24] Output [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] Input [13]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124, customer_id#146, year_total#147] -(120) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] Arguments: 100, [customer_id#46 ASC NULLS FIRST, customer_first_name#47 ASC NULLS FIRST, customer_last_name#48 ASC NULLS FIRST, customer_preferred_cust_flag#49 ASC NULLS FIRST, customer_birth_country#50 ASC NULLS FIRST, customer_login#51 ASC NULLS FIRST, customer_email_address#52 ASC NULLS FIRST], [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 -BroadcastExchange (124) -+- * ColumnarToRow (123) - +- CometFilter (122) - +- CometScan parquet spark_catalog.default.date_dim (121) +BroadcastExchange (112) ++- * ColumnarToRow (111) + +- CometFilter (110) + +- CometScan parquet spark_catalog.default.date_dim (109) (unknown) Scan parquet spark_catalog.default.date_dim @@ -699,22 +651,22 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(122) CometFilter +(110) CometFilter Input [2]: [d_date_sk#16, d_year#17] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(123) ColumnarToRow [codegen id : 1] +(111) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] -(124) BroadcastExchange +(112) BroadcastExchange Input [2]: [d_date_sk#16, d_year#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 -BroadcastExchange (128) -+- * ColumnarToRow (127) - +- CometFilter (126) - +- CometScan parquet spark_catalog.default.date_dim (125) +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 +BroadcastExchange (116) ++- * ColumnarToRow (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) (unknown) Scan parquet spark_catalog.default.date_dim @@ -724,23 +676,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(126) CometFilter +(114) CometFilter Input [2]: [d_date_sk#40, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) -(127) ColumnarToRow [codegen id : 1] +(115) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#40, d_year#41] -(128) BroadcastExchange +(116) BroadcastExchange Input [2]: [d_date_sk#40, d_year#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] -Subquery:3 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 62 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#39 +Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#39 -Subquery:5 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#114 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#114 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 103 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#39 +Subquery:6 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#39 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index efa385b428..99e255a0e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -11,11 +11,41 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - RowToColumnar - WholeStageCodegen (3) + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 + WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -26,166 +56,124 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #6 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 WholeStageCodegen (11) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - RowToColumnar - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #14 WholeStageCodegen (19) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 - RowToColumnar - WholeStageCodegen (18) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 + WholeStageCodegen (18) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + BroadcastExchange #16 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #17 WholeStageCodegen (23) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 - RowToColumnar - WholeStageCodegen (22) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 + WholeStageCodegen (22) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + BroadcastExchange #19 + WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt index 0a369db73d..fda4bbae0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt @@ -1,38 +1,37 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * HashAggregate (33) - +- * ColumnarToRow (32) - +- CometColumnarExchange (31) - +- RowToColumnar (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * ColumnarToRow (12) - : : : +- CometProject (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometColumnarExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (9) - : : : +- CometColumnarExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (5) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.warehouse (13) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- ReusedExchange (26) +TakeOrderedAndProject (33) ++- * HashAggregate (32) + +- Exchange (31) + +- * HashAggregate (30) + +- * Project (29) + +- * BroadcastHashJoin Inner BuildRight (28) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (13) + : : : +- * SortMergeJoin LeftOuter (12) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- * Sort (11) + : : : +- Exchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometProject (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (6) + : : +- BroadcastExchange (17) + : : +- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.warehouse (14) + : +- BroadcastExchange (24) + : +- * ColumnarToRow (23) + : +- CometProject (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- ReusedExchange (27) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -47,13 +46,16 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) -(7) CometProject +(8) CometProject Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -(8) CometColumnarExchange +(9) ColumnarToRow [codegen id : 3] Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(9) CometSort +(10) Exchange Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(10) CometSortMergeJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter +(11) Sort [codegen id : 4] +Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] +Arguments: [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST], false, 0 -(11) CometProject -Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] +(12) SortMergeJoin [codegen id : 8] +Left keys [2]: [cs_order_number#3, cs_item_sk#2] +Right keys [2]: [cr_order_number#8, cr_item_sk#7] +Join type: LeftOuter +Join condition: None -(12) ColumnarToRow [codegen id : 4] -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] +(13) Project [codegen id : 8] +Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] +Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] (unknown) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#11, w_state#12] @@ -97,24 +100,24 @@ Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(14) CometFilter +(15) CometFilter Input [2]: [w_warehouse_sk#11, w_state#12] Condition : isnotnull(w_warehouse_sk#11) -(15) ColumnarToRow [codegen id : 1] +(16) ColumnarToRow [codegen id : 5] Input [2]: [w_warehouse_sk#11, w_state#12] -(16) BroadcastExchange +(17) BroadcastExchange Input [2]: [w_warehouse_sk#11, w_state#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(17) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_warehouse_sk#1] Right keys [1]: [w_warehouse_sk#11] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(19) Project [codegen id : 8] Output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#12] @@ -125,79 +128,73 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] ReadSchema: struct -(20) CometFilter +(21) CometFilter Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] Condition : (((isnotnull(i_current_price#15) AND (i_current_price#15 >= 0.99)) AND (i_current_price#15 <= 1.49)) AND isnotnull(i_item_sk#13)) -(21) CometProject +(22) CometProject Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] Arguments: [i_item_sk#13, i_item_id#14], [i_item_sk#13, i_item_id#14] -(22) ColumnarToRow [codegen id : 2] +(23) ColumnarToRow [codegen id : 6] Input [2]: [i_item_sk#13, i_item_id#14] -(23) BroadcastExchange +(24) BroadcastExchange Input [2]: [i_item_sk#13, i_item_id#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(24) BroadcastHashJoin [codegen id : 4] +(25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#2] Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(25) Project [codegen id : 4] +(26) Project [codegen id : 8] Output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_sk#13, i_item_id#14] -(26) ReusedExchange [Reuses operator id: 38] +(27) ReusedExchange [Reuses operator id: 37] Output [2]: [d_date_sk#16, d_date#17] -(27) BroadcastHashJoin [codegen id : 4] +(28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#16] Join type: Inner Join condition: None -(28) Project [codegen id : 4] +(29) Project [codegen id : 8] Output [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date_sk#16, d_date#17] -(29) HashAggregate [codegen id : 4] +(30) HashAggregate [codegen id : 8] Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] Keys [2]: [w_state#12, i_item_id#14] Functions [2]: [partial_sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] Aggregate Attributes [4]: [sum#18, isEmpty#19, sum#20, isEmpty#21] Results [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] -(30) RowToColumnar -Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(31) CometColumnarExchange -Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(32) ColumnarToRow [codegen id : 5] +(31) Exchange Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(33) HashAggregate [codegen id : 5] +(32) HashAggregate [codegen id : 9] Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [2]: [w_state#12, i_item_id#14] Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27] Results [4]: [w_state#12, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26 AS sales_before#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27 AS sales_after#29] -(34) TakeOrderedAndProject +(33) TakeOrderedAndProject Input [4]: [w_state#12, i_item_id#14, sales_before#28, sales_after#29] Arguments: 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#28, sales_after#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometFilter (36) - +- CometScan parquet spark_catalog.default.date_dim (35) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) (unknown) Scan parquet spark_catalog.default.date_dim @@ -207,14 +204,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] ReadSchema: struct -(36) CometFilter +(35) CometFilter Input [2]: [d_date_sk#16, d_date#17] Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) -(37) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] -(38) BroadcastExchange +(37) BroadcastExchange Input [2]: [d_date_sk#16, d_date#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index cb386096a5..10e0735b4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -1,52 +1,60 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - WholeStageCodegen (5) + WholeStageCodegen (9) HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [w_state,i_item_id] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #2 - CometFilter [cs_warehouse_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_order_number,cr_item_sk] - CometColumnarExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + InputAdapter + Exchange [w_state,i_item_id] #1 + WholeStageCodegen (8) + HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + Project [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] + WholeStageCodegen (2) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [cs_warehouse_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) + BroadcastExchange #5 + WholeStageCodegen (5) ColumnarToRow InputAdapter - CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_item_id] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index ee6d6a04ec..7174e2746a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -1,26 +1,24 @@ == Physical Plan == -* ColumnarToRow (22) -+- CometTakeOrderedAndProject (21) - +- CometHashAggregate (20) - +- CometColumnarExchange (19) - +- RowToColumnar (18) - +- * HashAggregate (17) - +- * Project (16) - +- * BroadcastHashJoin Inner BuildRight (15) - :- * ColumnarToRow (4) - : +- CometProject (3) - : +- CometFilter (2) - : +- CometScan parquet spark_catalog.default.item (1) - +- BroadcastExchange (14) - +- * ColumnarToRow (13) - +- CometProject (12) - +- CometFilter (11) - +- CometHashAggregate (10) - +- CometColumnarExchange (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan parquet spark_catalog.default.item (5) +TakeOrderedAndProject (20) ++- * HashAggregate (19) + +- Exchange (18) + +- * HashAggregate (17) + +- * Project (16) + +- * BroadcastHashJoin Inner BuildRight (15) + :- * ColumnarToRow (4) + : +- CometProject (3) + : +- CometFilter (2) + : +- CometScan parquet spark_catalog.default.item (1) + +- BroadcastExchange (14) + +- * Project (13) + +- * Filter (12) + +- * HashAggregate (11) + +- Exchange (10) + +- * ColumnarToRow (9) + +- CometHashAggregate (8) + +- CometProject (7) + +- CometFilter (6) + +- CometScan parquet spark_catalog.default.item (5) (unknown) Scan parquet spark_catalog.default.item @@ -38,7 +36,7 @@ Condition : (((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_m Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] -(4) ColumnarToRow [codegen id : 2] +(4) ColumnarToRow [codegen id : 3] Input [2]: [i_manufact#2, i_product_name#3] (unknown) Scan parquet spark_catalog.default.item @@ -61,63 +59,61 @@ Input [1]: [i_manufact#5] Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -(9) CometColumnarExchange +(9) ColumnarToRow [codegen id : 1] Input [2]: [i_manufact#5, count#9] -Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) CometHashAggregate +(10) Exchange +Input [2]: [i_manufact#5, count#9] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(11) HashAggregate [codegen id : 2] Input [2]: [i_manufact#5, count#9] Keys [1]: [i_manufact#5] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#10] +Results [2]: [count(1)#10 AS item_cnt#11, i_manufact#5] -(11) CometFilter -Input [2]: [item_cnt#10, i_manufact#5] -Condition : (item_cnt#10 > 0) - -(12) CometProject -Input [2]: [item_cnt#10, i_manufact#5] -Arguments: [i_manufact#5], [i_manufact#5] +(12) Filter [codegen id : 2] +Input [2]: [item_cnt#11, i_manufact#5] +Condition : (item_cnt#11 > 0) -(13) ColumnarToRow [codegen id : 1] -Input [1]: [i_manufact#5] +(13) Project [codegen id : 2] +Output [1]: [i_manufact#5] +Input [2]: [item_cnt#11, i_manufact#5] (14) BroadcastExchange Input [1]: [i_manufact#5] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 2] +(15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] Right keys [1]: [i_manufact#5] Join type: Inner Join condition: None -(16) Project [codegen id : 2] +(16) Project [codegen id : 3] Output [1]: [i_product_name#3] Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] -(17) HashAggregate [codegen id : 2] +(17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] Keys [1]: [i_product_name#3] Functions: [] Aggregate Attributes: [] Results [1]: [i_product_name#3] -(18) RowToColumnar +(18) Exchange Input [1]: [i_product_name#3] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(19) CometColumnarExchange -Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometHashAggregate +(19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] Keys [1]: [i_product_name#3] Functions: [] +Aggregate Attributes: [] +Results [1]: [i_product_name#3] -(21) CometTakeOrderedAndProject -Input [1]: [i_product_name#3] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#3 ASC NULLS FIRST], output=[i_product_name#3]), 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] - -(22) ColumnarToRow [codegen id : 3] +(20) TakeOrderedAndProject Input [1]: [i_product_name#3] +Arguments: 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index 3c68188747..e31217066b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -1,28 +1,28 @@ -WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] - CometHashAggregate [i_product_name] - CometColumnarExchange [i_product_name] #1 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [i_product_name] - Project [i_product_name] - BroadcastHashJoin [i_manufact,i_manufact] - ColumnarToRow - InputAdapter - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] - CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow +TakeOrderedAndProject [i_product_name] + WholeStageCodegen (4) + HashAggregate [i_product_name] + InputAdapter + Exchange [i_product_name] #1 + WholeStageCodegen (3) + HashAggregate [i_product_name] + Project [i_product_name] + BroadcastHashJoin [i_manufact,i_manufact] + ColumnarToRow + InputAdapter + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact] + CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [i_manufact] + Filter [item_cnt] + HashAggregate [i_manufact,count] [count(1),item_cnt,count] InputAdapter - CometProject [i_manufact] - CometFilter [item_cnt] - CometHashAggregate [i_manufact,count] - CometColumnarExchange [i_manufact] #3 + Exchange [i_manufact] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometHashAggregate [i_manufact] CometProject [i_manufact] CometFilter [i_category,i_color,i_units,i_size,i_manufact] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt index 112e87bb78..958f358b7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt @@ -1,27 +1,25 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- * ColumnarToRow (21) - +- CometColumnarExchange (20) - +- RowToColumnar (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- Exchange (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) (unknown) Scan parquet spark_catalog.default.date_dim @@ -110,24 +108,18 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] -(19) RowToColumnar +(19) Exchange Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] +Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) CometColumnarExchange -Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) ColumnarToRow [codegen id : 4] -Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] - -(22) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 4] Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] Keys [3]: [d_year#2, i_category_id#8, i_category#9] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] Results [4]: [d_year#2, i_category_id#8, i_category#9, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum(ss_ext_sales_price)#14] -(23) TakeOrderedAndProject +(21) TakeOrderedAndProject Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] Arguments: 100, [sum(ss_ext_sales_price)#14 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt index 52aa2dbd5d..67906b8c7a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt @@ -1,33 +1,31 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (4) HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [d_year,i_category_id,i_category] #1 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] - Project [d_year,ss_ext_sales_price,i_category_id,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + Exchange [d_year,i_category_id,i_category] #1 + WholeStageCodegen (3) + HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_category_id,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_category_id,i_category] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt index 1a832bd08b..68b4b54005 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt @@ -1,27 +1,25 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- * ColumnarToRow (21) - +- CometColumnarExchange (20) - +- RowToColumnar (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.store (11) +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- Exchange (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.store (11) (unknown) Scan parquet spark_catalog.default.date_dim @@ -110,24 +108,18 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) T Aggregate Attributes [7]: [sum#11, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17] Results [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] -(19) RowToColumnar +(19) Exchange Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(s_store_name#9, s_store_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) CometColumnarExchange -Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(s_store_name#9, s_store_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) ColumnarToRow [codegen id : 4] -Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] - -(22) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 4] Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [2]: [s_store_name#9, s_store_id#8] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))] Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31] Results [9]: [s_store_name#9, s_store_id#8, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25,17,2) AS sun_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26,17,2) AS mon_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27,17,2) AS tue_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28,17,2) AS wed_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29,17,2) AS thu_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30,17,2) AS fri_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31,17,2) AS sat_sales#38] -(23) TakeOrderedAndProject +(21) TakeOrderedAndProject Input [9]: [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] Arguments: 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#32 ASC NULLS FIRST, mon_sales#33 ASC NULLS FIRST, tue_sales#34 ASC NULLS FIRST, wed_sales#35 ASC NULLS FIRST, thu_sales#36 ASC NULLS FIRST, fri_sales#37 ASC NULLS FIRST, sat_sales#38 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt index 2fc9968cc0..ef20430969 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt @@ -1,33 +1,31 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] WholeStageCodegen (4) HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_store_name,s_store_id] #1 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,ss_sales_price,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [d_day_name,ss_store_sk,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + InputAdapter + Exchange [s_store_name,s_store_id] #1 + WholeStageCodegen (3) + HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [d_day_name,ss_sales_price,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [d_day_name,ss_store_sk,ss_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_day_name] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,s_store_id,s_store_name] + CometFilter [s_gmt_offset,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index e3a07a650d..7a7d667378 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -1,37 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (23) - : : +- * SortMergeJoin Inner (22) +TakeOrderedAndProject (32) ++- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) + : :- * Project (22) + : : +- * SortMergeJoin Inner (21) : : :- * Sort (14) : : : +- * Project (13) : : : +- * Filter (12) : : : +- Window (11) - : : : +- * ColumnarToRow (10) - : : : +- CometSort (9) - : : : +- CometColumnarExchange (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometColumnarExchange (5) + : : : +- * Sort (10) + : : : +- Exchange (9) + : : : +- * Filter (8) + : : : +- * HashAggregate (7) + : : : +- Exchange (6) + : : : +- * ColumnarToRow (5) : : : +- CometHashAggregate (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- * Sort (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- Window (18) - : : +- * ColumnarToRow (17) - : : +- CometSort (16) - : : +- ReusedExchange (15) - : +- BroadcastExchange (27) - : +- * ColumnarToRow (26) - : +- CometFilter (25) - : +- CometScan parquet spark_catalog.default.item (24) - +- ReusedExchange (30) + : : +- * Sort (20) + : : +- * Project (19) + : : +- * Filter (18) + : : +- Window (17) + : : +- * Sort (16) + : : +- ReusedExchange (15) + : +- BroadcastExchange (26) + : +- * ColumnarToRow (25) + : +- CometFilter (24) + : +- CometScan parquet spark_catalog.default.item (23) + +- ReusedExchange (29) (unknown) Scan parquet spark_catalog.default.store_sales @@ -54,169 +53,170 @@ Input [2]: [ss_item_sk#1, ss_net_profit#3] Keys [1]: [ss_item_sk#1] Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -(5) CometColumnarExchange +(5) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(6) CometHashAggregate +(6) Exchange +Input [3]: [ss_item_sk#1, sum#5, count#6] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(7) HashAggregate [codegen id : 2] Input [3]: [ss_item_sk#1, sum#5, count#6] Keys [1]: [ss_item_sk#1] Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#7] +Results [2]: [ss_item_sk#1 AS item_sk#8, cast((avg(UnscaledValue(ss_net_profit#3))#7 / 100.0) as decimal(11,6)) AS rank_col#9] -(7) CometFilter -Input [2]: [item_sk#7, rank_col#8] -Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) +(8) Filter [codegen id : 2] +Input [2]: [item_sk#8, rank_col#9] +Condition : (isnotnull(rank_col#9) AND (cast(rank_col#9 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#10, [id=#11]))) -(8) CometColumnarExchange -Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(9) Exchange +Input [2]: [item_sk#8, rank_col#9] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(9) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(10) ColumnarToRow [codegen id : 1] -Input [2]: [item_sk#7, rank_col#8] +(10) Sort [codegen id : 3] +Input [2]: [item_sk#8, rank_col#9] +Arguments: [rank_col#9 ASC NULLS FIRST], false, 0 (11) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(12) Filter [codegen id : 2] -Input [3]: [item_sk#7, rank_col#8, rnk#11] -Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) +Input [2]: [item_sk#8, rank_col#9] +Arguments: [rank(rank_col#9) windowspecdefinition(rank_col#9 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#12], [rank_col#9 ASC NULLS FIRST] -(13) Project [codegen id : 2] -Output [2]: [item_sk#7, rnk#11] -Input [3]: [item_sk#7, rank_col#8, rnk#11] +(12) Filter [codegen id : 4] +Input [3]: [item_sk#8, rank_col#9, rnk#12] +Condition : ((rnk#12 < 11) AND isnotnull(item_sk#8)) -(14) Sort [codegen id : 2] -Input [2]: [item_sk#7, rnk#11] -Arguments: [rnk#11 ASC NULLS FIRST], false, 0 +(13) Project [codegen id : 4] +Output [2]: [item_sk#8, rnk#12] +Input [3]: [item_sk#8, rank_col#9, rnk#12] -(15) ReusedExchange [Reuses operator id: 8] -Output [2]: [item_sk#12, rank_col#13] +(14) Sort [codegen id : 4] +Input [2]: [item_sk#8, rnk#12] +Arguments: [rnk#12 ASC NULLS FIRST], false, 0 -(16) CometSort -Input [2]: [item_sk#12, rank_col#13] -Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] +(15) ReusedExchange [Reuses operator id: 9] +Output [2]: [item_sk#13, rank_col#14] -(17) ColumnarToRow [codegen id : 3] -Input [2]: [item_sk#12, rank_col#13] +(16) Sort [codegen id : 7] +Input [2]: [item_sk#13, rank_col#14] +Arguments: [rank_col#14 DESC NULLS LAST], false, 0 -(18) Window -Input [2]: [item_sk#12, rank_col#13] -Arguments: [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] +(17) Window +Input [2]: [item_sk#13, rank_col#14] +Arguments: [rank(rank_col#14) windowspecdefinition(rank_col#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#15], [rank_col#14 DESC NULLS LAST] -(19) Filter [codegen id : 4] -Input [3]: [item_sk#12, rank_col#13, rnk#14] -Condition : ((rnk#14 < 11) AND isnotnull(item_sk#12)) +(18) Filter [codegen id : 8] +Input [3]: [item_sk#13, rank_col#14, rnk#15] +Condition : ((rnk#15 < 11) AND isnotnull(item_sk#13)) -(20) Project [codegen id : 4] -Output [2]: [item_sk#12, rnk#14] -Input [3]: [item_sk#12, rank_col#13, rnk#14] +(19) Project [codegen id : 8] +Output [2]: [item_sk#13, rnk#15] +Input [3]: [item_sk#13, rank_col#14, rnk#15] -(21) Sort [codegen id : 4] -Input [2]: [item_sk#12, rnk#14] -Arguments: [rnk#14 ASC NULLS FIRST], false, 0 +(20) Sort [codegen id : 8] +Input [2]: [item_sk#13, rnk#15] +Arguments: [rnk#15 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 7] -Left keys [1]: [rnk#11] -Right keys [1]: [rnk#14] +(21) SortMergeJoin [codegen id : 11] +Left keys [1]: [rnk#12] +Right keys [1]: [rnk#15] Join type: Inner Join condition: None -(23) Project [codegen id : 7] -Output [3]: [item_sk#7, rnk#11, item_sk#12] -Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] +(22) Project [codegen id : 11] +Output [3]: [item_sk#8, rnk#12, item_sk#13] +Input [4]: [item_sk#8, rnk#12, item_sk#13, rnk#15] (unknown) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_product_name#16] +Output [2]: [i_item_sk#16, i_product_name#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [i_item_sk#15, i_product_name#16] -Condition : isnotnull(i_item_sk#15) +(24) CometFilter +Input [2]: [i_item_sk#16, i_product_name#17] +Condition : isnotnull(i_item_sk#16) -(26) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#15, i_product_name#16] +(25) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_sk#16, i_product_name#17] -(27) BroadcastExchange -Input [2]: [i_item_sk#15, i_product_name#16] +(26) BroadcastExchange +Input [2]: [i_item_sk#16, i_product_name#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(28) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#15] +(27) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [item_sk#8] +Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(29) Project [codegen id : 7] -Output [3]: [rnk#11, item_sk#12, i_product_name#16] -Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#16] +(28) Project [codegen id : 11] +Output [3]: [rnk#12, item_sk#13, i_product_name#17] +Input [5]: [item_sk#8, rnk#12, item_sk#13, i_item_sk#16, i_product_name#17] -(30) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#17, i_product_name#18] +(29) ReusedExchange [Reuses operator id: 26] +Output [2]: [i_item_sk#18, i_product_name#19] -(31) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#12] -Right keys [1]: [i_item_sk#17] +(30) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [item_sk#13] +Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 7] -Output [3]: [rnk#11, i_product_name#16 AS best_performing#19, i_product_name#18 AS worst_performing#20] -Input [5]: [rnk#11, item_sk#12, i_product_name#16, i_item_sk#17, i_product_name#18] +(31) Project [codegen id : 11] +Output [3]: [rnk#12, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] +Input [5]: [rnk#12, item_sk#13, i_product_name#17, i_item_sk#18, i_product_name#19] -(33) TakeOrderedAndProject -Input [3]: [rnk#11, best_performing#19, worst_performing#20] -Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#19, worst_performing#20] +(32) TakeOrderedAndProject +Input [3]: [rnk#12, best_performing#20, worst_performing#21] +Arguments: 100, [rnk#12 ASC NULLS FIRST], [rnk#12, best_performing#20, worst_performing#21] ===== Subqueries ===== -Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* ColumnarToRow (40) -+- CometHashAggregate (39) - +- CometColumnarExchange (38) - +- CometHashAggregate (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.store_sales (34) +Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#10, [id=#11] +* HashAggregate (39) ++- Exchange (38) + +- * ColumnarToRow (37) + +- CometHashAggregate (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.store_sales (33) (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#21, ss_store_sk#22, ss_net_profit#23, ss_sold_date_sk#24] +Output [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(35) CometFilter -Input [4]: [ss_addr_sk#21, ss_store_sk#22, ss_net_profit#23, ss_sold_date_sk#24] -Condition : ((isnotnull(ss_store_sk#22) AND (ss_store_sk#22 = 4)) AND isnull(ss_addr_sk#21)) +(34) CometFilter +Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] +Condition : ((isnotnull(ss_store_sk#23) AND (ss_store_sk#23 = 4)) AND isnull(ss_addr_sk#22)) -(36) CometProject -Input [4]: [ss_addr_sk#21, ss_store_sk#22, ss_net_profit#23, ss_sold_date_sk#24] -Arguments: [ss_store_sk#22, ss_net_profit#23], [ss_store_sk#22, ss_net_profit#23] +(35) CometProject +Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] +Arguments: [ss_store_sk#23, ss_net_profit#24], [ss_store_sk#23, ss_net_profit#24] -(37) CometHashAggregate -Input [2]: [ss_store_sk#22, ss_net_profit#23] -Keys [1]: [ss_store_sk#22] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#23))] +(36) CometHashAggregate +Input [2]: [ss_store_sk#23, ss_net_profit#24] +Keys [1]: [ss_store_sk#23] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#24))] -(38) CometColumnarExchange -Input [3]: [ss_store_sk#22, sum#25, count#26] -Arguments: hashpartitioning(ss_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(37) ColumnarToRow [codegen id : 1] +Input [3]: [ss_store_sk#23, sum#26, count#27] -(39) CometHashAggregate -Input [3]: [ss_store_sk#22, sum#25, count#26] -Keys [1]: [ss_store_sk#22] -Functions [1]: [avg(UnscaledValue(ss_net_profit#23))] +(38) Exchange +Input [3]: [ss_store_sk#23, sum#26, count#27] +Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(40) ColumnarToRow [codegen id : 1] -Input [1]: [rank_col#27] +(39) HashAggregate [codegen id : 2] +Input [3]: [ss_store_sk#23, sum#26, count#27] +Keys [1]: [ss_store_sk#23] +Functions [1]: [avg(UnscaledValue(ss_net_profit#24))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#24))#28] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#24))#28 / 100.0) as decimal(11,6)) AS rank_col#29] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index faccc1ec5e..701a909122 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (7) + WholeStageCodegen (11) Project [rnk,i_product_name,i_product_name] BroadcastHashJoin [item_sk,i_item_sk] Project [rnk,item_sk,i_product_name] @@ -7,49 +7,54 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Project [item_sk,rnk,item_sk] SortMergeJoin [rnk,rnk] InputAdapter - WholeStageCodegen (2) + WholeStageCodegen (4) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] - WholeStageCodegen (1) - ColumnarToRow + WholeStageCodegen (3) + Sort [rank_col] InputAdapter - CometSort [rank_col] - CometColumnarExchange #1 - CometFilter [rank_col] + Exchange #1 + WholeStageCodegen (2) + Filter [rank_col] Subquery #1 - WholeStageCodegen (1) - ColumnarToRow + WholeStageCodegen (2) + HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count] InputAdapter - CometHashAggregate [ss_store_sk,sum,count] - CometColumnarExchange [ss_store_sk] #3 - CometHashAggregate [ss_store_sk,ss_net_profit] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [ss_item_sk,sum,count] - CometColumnarExchange [ss_item_sk] #2 - CometHashAggregate [ss_item_sk,ss_net_profit] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + Exchange [ss_store_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_store_sk,ss_net_profit] + CometProject [ss_store_sk,ss_net_profit] + CometFilter [ss_store_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] + InputAdapter + Exchange [ss_item_sk] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_item_sk,ss_net_profit] + CometProject [ss_item_sk,ss_net_profit] + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (8) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] - WholeStageCodegen (3) - ColumnarToRow + WholeStageCodegen (7) + Sort [rank_col] InputAdapter - CometSort [rank_col] - ReusedExchange [item_sk,rank_col] #1 + ReusedExchange [item_sk,rank_col] #1 InputAdapter BroadcastExchange #4 - WholeStageCodegen (5) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometFilter [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index d965422219..d5d860a393 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -1,42 +1,40 @@ == Physical Plan == -TakeOrderedAndProject (38) -+- * HashAggregate (37) - +- * ColumnarToRow (36) - +- CometColumnarExchange (35) - +- RowToColumnar (34) - +- * HashAggregate (33) - +- * Project (32) - +- * Filter (31) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.customer (4) - : : : +- BroadcastExchange (13) - : : : +- * ColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (16) - : +- BroadcastExchange (22) - : +- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.item (25) +TakeOrderedAndProject (36) ++- * HashAggregate (35) + +- Exchange (34) + +- * HashAggregate (33) + +- * Project (32) + +- * Filter (31) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.customer (4) + : : : +- BroadcastExchange (13) + : : : +- * ColumnarToRow (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.customer_address (10) + : : +- ReusedExchange (16) + : +- BroadcastExchange (22) + : +- * ColumnarToRow (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.item (25) (unknown) Scan parquet spark_catalog.default.web_sales @@ -110,7 +108,7 @@ Join condition: None Output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11] Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#11] -(16) ReusedExchange [Reuses operator id: 43] +(16) ReusedExchange [Reuses operator id: 41] Output [1]: [d_date_sk#12] (17) BroadcastHashJoin [codegen id : 6] @@ -194,35 +192,29 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] Aggregate Attributes [1]: [sum#17] Results [3]: [ca_zip#11, ca_city#10, sum#18] -(34) RowToColumnar +(34) Exchange Input [3]: [ca_zip#11, ca_city#10, sum#18] +Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(35) CometColumnarExchange -Input [3]: [ca_zip#11, ca_city#10, sum#18] -Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) ColumnarToRow [codegen id : 7] -Input [3]: [ca_zip#11, ca_city#10, sum#18] - -(37) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 7] Input [3]: [ca_zip#11, ca_city#10, sum#18] Keys [2]: [ca_zip#11, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#19] Results [3]: [ca_zip#11, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#19,17,2) AS sum(ws_sales_price)#20] -(38) TakeOrderedAndProject +(36) TakeOrderedAndProject Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (43) -+- * ColumnarToRow (42) - +- CometProject (41) - +- CometFilter (40) - +- CometScan parquet spark_catalog.default.date_dim (39) +BroadcastExchange (41) ++- * ColumnarToRow (40) + +- CometProject (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.date_dim (37) (unknown) Scan parquet spark_catalog.default.date_dim @@ -232,18 +224,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(40) CometFilter +(38) CometFilter Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] Condition : ((((isnotnull(d_qoy#22) AND isnotnull(d_year#21)) AND (d_qoy#22 = 2)) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#12)) -(41) CometProject +(39) CometProject Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] Arguments: [d_date_sk#12], [d_date_sk#12] -(42) ColumnarToRow [codegen id : 1] +(40) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#12] -(43) BroadcastExchange +(41) BroadcastExchange Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index 05022ce60c..383cbb7e3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -1,63 +1,61 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] WholeStageCodegen (7) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip,ca_city] #1 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - Project [ws_sales_price,ca_city,ca_zip,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + Exchange [ca_zip,ca_city] #1 + WholeStageCodegen (6) + HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] + Project [ws_sales_price,ca_city,ca_zip] + Filter [ca_zip,exists] + BroadcastHashJoin [i_item_id,i_item_id] + Project [ws_sales_price,ca_city,ca_zip,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [i_item_id] - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [i_item_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index 3fb39c6df1..84d65306e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -1,45 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * HashAggregate (31) - : : +- * ColumnarToRow (30) - : : +- CometColumnarExchange (29) - : : +- RowToColumnar (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (35) - : +- * ColumnarToRow (34) - : +- CometFilter (33) - : +- CometScan parquet spark_catalog.default.customer (32) - +- ReusedExchange (38) +TakeOrderedAndProject (39) ++- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * HashAggregate (29) + : : +- Exchange (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * Project (13) + : : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : : :- * Project (6) + : : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- ReusedExchange (4) + : : : : +- BroadcastExchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store (7) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.household_demographics (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.customer_address (21) + : +- BroadcastExchange (33) + : +- * ColumnarToRow (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.customer (30) + +- ReusedExchange (36) (unknown) Scan parquet spark_catalog.default.store_sales @@ -57,7 +55,7 @@ Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotn (3) ColumnarToRow [codegen id : 5] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -(4) ReusedExchange [Reuses operator id: 46] +(4) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#10] (5) BroadcastHashJoin [codegen id : 5] @@ -169,17 +167,11 @@ Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(Unscale Aggregate Attributes [2]: [sum#18, sum#19] Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -(28) RowToColumnar +(28) Exchange Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) CometColumnarExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) ColumnarToRow [codegen id : 8] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] - -(31) HashAggregate [codegen id : 8] +(29) HashAggregate [codegen id : 8] Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] @@ -193,52 +185,52 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(33) CometFilter +(31) CometFilter Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) -(34) ColumnarToRow [codegen id : 6] +(32) ColumnarToRow [codegen id : 6] Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -(35) BroadcastExchange +(33) BroadcastExchange Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#27] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(35) Project [codegen id : 8] Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#29, c_last_name#30] Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -(38) ReusedExchange [Reuses operator id: 24] +(36) ReusedExchange [Reuses operator id: 24] Output [2]: [ca_address_sk#31, ca_city#32] -(39) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#28] Right keys [1]: [ca_address_sk#31] Join type: Inner Join condition: NOT (ca_city#32 = bought_city#24) -(40) Project [codegen id : 8] +(38) Project [codegen id : 8] Output [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_city#32] -(41) TakeOrderedAndProject +(39) TakeOrderedAndProject Input [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] Arguments: 100, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#24 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (46) -+- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.date_dim (42) +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) (unknown) Scan parquet spark_catalog.default.date_dim @@ -248,18 +240,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter +(41) CometFilter Input [3]: [d_date_sk#10, d_year#33, d_dow#34] Condition : ((d_dow#34 IN (6,0) AND d_year#33 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(44) CometProject +(42) CometProject Input [3]: [d_date_sk#10, d_year#33, d_dow#34] Arguments: [d_date_sk#10], [d_date_sk#10] -(45) ColumnarToRow [codegen id : 1] +(43) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(46) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index f40df7e979..04c59a2d35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -5,57 +5,55 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + WholeStageCodegen (5) + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dow,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometProject [s_store_sk] + CometFilter [s_city,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index d12c4ab567..9ea57de066 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -1,57 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (53) -+- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (32) - : : +- * Filter (31) - : : +- Window (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * ColumnarToRow (27) - : : +- CometSort (26) - : : +- CometColumnarExchange (25) - : : +- RowToColumnar (24) - : : +- * HashAggregate (23) - : : +- * ColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- RowToColumnar (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (42) - : +- * Project (41) - : +- Window (40) - : +- * ColumnarToRow (39) - : +- CometSort (38) - : +- CometColumnarExchange (37) - : +- RowToColumnar (36) - : +- * HashAggregate (35) - : +- * ColumnarToRow (34) - : +- ReusedExchange (33) - +- BroadcastExchange (50) - +- * Project (49) - +- Window (48) - +- * ColumnarToRow (47) - +- CometSort (46) - +- ReusedExchange (45) +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * Filter (25) + : : +- Window (24) + : : +- * Sort (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- Exchange (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) (unknown) Scan parquet spark_catalog.default.item @@ -97,7 +89,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 57] +(10) ReusedExchange [Reuses operator id: 49] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -145,149 +137,125 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [1]: [sum#15] Results [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -(20) RowToColumnar +(20) Exchange Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometColumnarExchange -Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) ColumnarToRow [codegen id : 5] -Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] - -(23) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 5] Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#17] Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS _w0#19] -(24) RowToColumnar -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] - -(25) CometColumnarExchange +(22) Exchange Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(26) CometSort +(23) Sort [codegen id : 6] Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 -(27) ColumnarToRow [codegen id : 6] -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] - -(28) Window +(24) Window Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(29) Filter [codegen id : 7] +(25) Filter [codegen id : 7] Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(30) Window +(26) Window Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] -(31) Filter [codegen id : 22] +(27) Filter [codegen id : 22] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(32) Project [codegen id : 22] +(28) Project [codegen id : 22] Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -(33) ReusedExchange [Reuses operator id: 21] +(29) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(34) ColumnarToRow [codegen id : 12] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(35) HashAggregate [codegen id : 12] +(30) HashAggregate [codegen id : 12] Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] -(36) RowToColumnar -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] - -(37) CometColumnarExchange +(31) Exchange Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) CometSort +(32) Sort [codegen id : 13] Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 -(39) ColumnarToRow [codegen id : 13] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] - -(40) Window +(33) Window Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(41) Project [codegen id : 14] +(34) Project [codegen id : 14] Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] -(42) BroadcastExchange +(35) BroadcastExchange Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 22] +(36) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] +(37) Project [codegen id : 22] Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -(45) ReusedExchange [Reuses operator id: 37] +(38) ReusedExchange [Reuses operator id: 31] Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -(46) CometSort +(39) Sort [codegen id : 20] Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 -(47) ColumnarToRow [codegen id : 20] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] - -(48) Window +(40) Window Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(49) Project [codegen id : 21] +(41) Project [codegen id : 21] Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] -(50) BroadcastExchange +(42) BroadcastExchange Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 22] +(43) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] Join type: Inner Join condition: None -(52) Project [codegen id : 22] +(44) Project [codegen id : 22] Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -(53) TakeOrderedAndProject +(45) TakeOrderedAndProject Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (57) -+- * ColumnarToRow (56) - +- CometFilter (55) - +- CometScan parquet spark_catalog.default.date_dim (54) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim @@ -297,14 +265,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(55) CometFilter +(47) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(56) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(57) BroadcastExchange +(49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index 3640cb9bac..80b8da7b14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -13,52 +13,48 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (6) - ColumnarToRow + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 WholeStageCodegen (14) @@ -66,16 +62,13 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (13) - ColumnarToRow + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - RowToColumnar - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + Exchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 WholeStageCodegen (21) @@ -83,7 +76,6 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (20) - ColumnarToRow + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt index ddceb24b2d..a13a86a914 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt @@ -1,34 +1,32 @@ == Physical Plan == -* HashAggregate (30) -+- * ColumnarToRow (29) - +- CometColumnarExchange (28) - +- RowToColumnar (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.customer_demographics (10) - : +- BroadcastExchange (20) - : +- * ColumnarToRow (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.customer_address (16) - +- ReusedExchange (23) +* HashAggregate (28) ++- Exchange (27) + +- * HashAggregate (26) + +- * Project (25) + +- * BroadcastHashJoin Inner BuildRight (24) + :- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.customer_demographics (10) + : +- BroadcastExchange (20) + : +- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.customer_address (16) + +- ReusedExchange (23) (unknown) Scan parquet spark_catalog.default.store_sales @@ -134,7 +132,7 @@ Join condition: ((((ca_state#14 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND Output [2]: [ss_quantity#4, ss_sold_date_sk#7] Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#13, ca_state#14] -(23) ReusedExchange [Reuses operator id: 35] +(23) ReusedExchange [Reuses operator id: 33] Output [1]: [d_date_sk#16] (24) BroadcastHashJoin [codegen id : 5] @@ -154,17 +152,11 @@ Functions [1]: [partial_sum(ss_quantity#4)] Aggregate Attributes [1]: [sum#17] Results [1]: [sum#18] -(27) RowToColumnar +(27) Exchange Input [1]: [sum#18] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(28) CometColumnarExchange -Input [1]: [sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) ColumnarToRow [codegen id : 6] -Input [1]: [sum#18] - -(30) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 6] Input [1]: [sum#18] Keys: [] Functions [1]: [sum(ss_quantity#4)] @@ -174,11 +166,11 @@ Results [1]: [sum(ss_quantity#4)#19 AS sum(ss_quantity)#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) (unknown) Scan parquet spark_catalog.default.date_dim @@ -188,18 +180,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter +(30) CometFilter Input [2]: [d_date_sk#16, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#16)) -(33) CometProject +(31) CometProject Input [2]: [d_date_sk#16, d_year#21] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) ColumnarToRow [codegen id : 1] +(32) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] -(35) BroadcastExchange +(33) BroadcastExchange Input [1]: [d_date_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt index bf76932af0..4022da74f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt @@ -1,52 +1,50 @@ WholeStageCodegen (6) HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [ss_quantity] [sum,sum] - Project [ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] - CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + Exchange #1 + WholeStageCodegen (5) + HashAggregate [ss_quantity] [sum,sum] + Project [ss_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sold_date_sk] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Project [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 7ec19d4046..3e14838b85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -1,92 +1,78 @@ == Physical Plan == -* ColumnarToRow (88) -+- CometTakeOrderedAndProject (87) - +- CometHashAggregate (86) - +- CometColumnarExchange (85) - +- RowToColumnar (84) - +- * HashAggregate (83) - +- Union (82) - :- * Project (27) - : +- * Filter (26) - : +- Window (25) - : +- * Sort (24) - : +- Window (23) - : +- * ColumnarToRow (22) - : +- CometSort (21) - : +- CometColumnarExchange (20) - : +- RowToColumnar (19) - : +- * HashAggregate (18) - : +- * ColumnarToRow (17) - : +- CometColumnarExchange (16) - : +- RowToColumnar (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.web_returns (5) - : +- ReusedExchange (11) - :- * Project (54) - : +- * Filter (53) - : +- Window (52) - : +- * Sort (51) - : +- Window (50) - : +- * ColumnarToRow (49) - : +- CometSort (48) - : +- CometColumnarExchange (47) - : +- RowToColumnar (46) - : +- * HashAggregate (45) - : +- * ColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- RowToColumnar (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * ColumnarToRow (37) - : : +- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometBroadcastExchange (31) - : : : +- CometProject (30) - : : : +- CometFilter (29) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometScan parquet spark_catalog.default.catalog_returns (32) - : +- ReusedExchange (38) - +- * Project (81) - +- * Filter (80) - +- Window (79) - +- * Sort (78) - +- Window (77) - +- * ColumnarToRow (76) - +- CometSort (75) - +- CometColumnarExchange (74) - +- RowToColumnar (73) - +- * HashAggregate (72) - +- * ColumnarToRow (71) - +- CometColumnarExchange (70) - +- RowToColumnar (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * ColumnarToRow (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometBroadcastExchange (58) - : : +- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan parquet spark_catalog.default.store_sales (55) - : +- CometProject (61) - : +- CometFilter (60) - : +- CometScan parquet spark_catalog.default.store_returns (59) - +- ReusedExchange (65) +TakeOrderedAndProject (74) ++- * HashAggregate (73) + +- Exchange (72) + +- * HashAggregate (71) + +- Union (70) + :- * Project (23) + : +- * Filter (22) + : +- Window (21) + : +- * Sort (20) + : +- Window (19) + : +- * Sort (18) + : +- Exchange (17) + : +- * HashAggregate (16) + : +- Exchange (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- ReusedExchange (11) + :- * Project (46) + : +- * Filter (45) + : +- Window (44) + : +- * Sort (43) + : +- Window (42) + : +- * Sort (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * ColumnarToRow (33) + : : +- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.catalog_returns (28) + : +- ReusedExchange (34) + +- * Project (69) + +- * Filter (68) + +- Window (67) + +- * Sort (66) + +- Window (65) + +- * Sort (64) + +- Exchange (63) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * ColumnarToRow (56) + : +- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometBroadcastExchange (50) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan parquet spark_catalog.default.store_sales (47) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.store_returns (51) + +- ReusedExchange (57) (unknown) Scan parquet spark_catalog.default.web_sales @@ -136,7 +122,7 @@ Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_re (10) ColumnarToRow [codegen id : 2] Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(11) ReusedExchange [Reuses operator id: 93] +(11) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#13] (12) BroadcastHashJoin [codegen id : 2] @@ -156,54 +142,42 @@ Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coa Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(15) RowToColumnar +(15) Exchange Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(16) CometColumnarExchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(17) ColumnarToRow [codegen id : 3] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(18) HashAggregate [codegen id : 3] +(16) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] -(19) RowToColumnar +(17) Exchange Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(20) CometColumnarExchange +(18) Sort [codegen id : 4] Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 -(21) CometSort -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] - -(22) ColumnarToRow [codegen id : 4] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] - -(23) Window +(19) Window Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(24) Sort [codegen id : 5] +(20) Sort [codegen id : 5] Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 -(25) Window +(21) Window Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(26) Filter [codegen id : 6] +(22) Filter [codegen id : 6] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(27) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] @@ -215,15 +189,15 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(30) CometProject +(26) CometProject Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -(31) CometBroadcastExchange +(27) CometBroadcastExchange Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] @@ -234,94 +208,82 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(33) CometFilter +(29) CometFilter Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) -(34) CometProject +(30) CometProject Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -(35) CometBroadcastHashJoin +(31) CometBroadcastHashJoin Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner -(36) CometProject +(32) CometProject Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -(37) ColumnarToRow [codegen id : 8] +(33) ColumnarToRow [codegen id : 8] Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -(38) ReusedExchange [Reuses operator id: 93] +(34) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#48] -(39) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#41] Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(40) Project [codegen id : 8] +(36) Project [codegen id : 8] Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] -(41) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 8] Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] Keys [1]: [cs_item_sk#36] Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -(42) RowToColumnar -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] - -(43) CometColumnarExchange -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(44) ColumnarToRow [codegen id : 9] +(38) Exchange Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(45) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 9] Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] Keys [1]: [cs_item_sk#36] Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] -(46) RowToColumnar -Input [3]: [item#65, return_ratio#66, currency_ratio#67] - -(47) CometColumnarExchange -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(48) CometSort +(40) Exchange Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [item#65, return_ratio#66, currency_ratio#67], [return_ratio#66 ASC NULLS FIRST] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(49) ColumnarToRow [codegen id : 10] +(41) Sort [codegen id : 10] Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 -(50) Window +(42) Window Input [3]: [item#65, return_ratio#66, currency_ratio#67] Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] -(51) Sort [codegen id : 11] +(43) Sort [codegen id : 11] Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 -(52) Window +(44) Window Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] -(53) Filter [codegen id : 12] +(45) Filter [codegen id : 12] Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) -(54) Project [codegen id : 12] +(46) Project [codegen id : 12] Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] @@ -333,15 +295,15 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(56) CometFilter +(48) CometFilter Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) -(57) CometProject +(49) CometProject Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -(58) CometBroadcastExchange +(50) CometBroadcastExchange Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] @@ -352,133 +314,117 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(60) CometFilter +(52) CometFilter Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) -(61) CometProject +(53) CometProject Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -(62) CometBroadcastHashJoin +(54) CometBroadcastHashJoin Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner -(63) CometProject +(55) CometProject Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -(64) ColumnarToRow [codegen id : 14] +(56) ColumnarToRow [codegen id : 14] Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -(65) ReusedExchange [Reuses operator id: 93] +(57) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#83] -(66) BroadcastHashJoin [codegen id : 14] +(58) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_sold_date_sk#76] Right keys [1]: [d_date_sk#83] Join type: Inner Join condition: None -(67) Project [codegen id : 14] +(59) Project [codegen id : 14] Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] -(68) HashAggregate [codegen id : 14] +(60) HashAggregate [codegen id : 14] Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] Keys [1]: [ss_item_sk#71] Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -(69) RowToColumnar -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(70) CometColumnarExchange +(61) Exchange Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(71) ColumnarToRow [codegen id : 15] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(72) HashAggregate [codegen id : 15] +(62) HashAggregate [codegen id : 15] Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] Keys [1]: [ss_item_sk#71] Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] -(73) RowToColumnar -Input [3]: [item#100, return_ratio#101, currency_ratio#102] - -(74) CometColumnarExchange +(63) Exchange Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(75) CometSort +(64) Sort [codegen id : 16] Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [item#100, return_ratio#101, currency_ratio#102], [return_ratio#101 ASC NULLS FIRST] +Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 -(76) ColumnarToRow [codegen id : 16] -Input [3]: [item#100, return_ratio#101, currency_ratio#102] - -(77) Window +(65) Window Input [3]: [item#100, return_ratio#101, currency_ratio#102] Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] -(78) Sort [codegen id : 17] +(66) Sort [codegen id : 17] Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 -(79) Window +(67) Window Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] -(80) Filter [codegen id : 18] +(68) Filter [codegen id : 18] Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) -(81) Project [codegen id : 18] +(69) Project [codegen id : 18] Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -(82) Union +(70) Union -(83) HashAggregate [codegen id : 19] +(71) HashAggregate [codegen id : 19] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(84) RowToColumnar -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(85) CometColumnarExchange +(72) Exchange Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(86) CometHashAggregate +(73) HashAggregate [codegen id : 20] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(87) CometTakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(88) ColumnarToRow [codegen id : 20] +(74) TakeOrderedAndProject Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (93) -+- * ColumnarToRow (92) - +- CometProject (91) - +- CometFilter (90) - +- CometScan parquet spark_catalog.default.date_dim (89) +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan parquet spark_catalog.default.date_dim (75) (unknown) Scan parquet spark_catalog.default.date_dim @@ -488,23 +434,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(90) CometFilter +(76) CometFilter Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) -(91) CometProject +(77) CometProject Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Arguments: [d_date_sk#13], [d_date_sk#13] -(92) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(93) BroadcastExchange +(79) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index 42f84a6e7f..43ebf34cc0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -1,135 +1,121 @@ -WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - RowToColumnar - WholeStageCodegen (19) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometSort [return_ratio] - CometColumnarExchange #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (12) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (11) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometSort [return_ratio] - CometColumnarExchange #6 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #8 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (18) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (17) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometSort [return_ratio] - CometColumnarExchange #9 - RowToColumnar - WholeStageCodegen (15) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - RowToColumnar - WholeStageCodegen (14) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #11 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 +TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] + WholeStageCodegen (20) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (19) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + Sort [return_ratio] + InputAdapter + Exchange #2 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (12) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (11) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (10) + Sort [return_ratio] + InputAdapter + Exchange #6 + WholeStageCodegen (9) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #7 + WholeStageCodegen (8) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #8 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (18) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (17) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (16) + Sort [return_ratio] + InputAdapter + Exchange #9 + WholeStageCodegen (15) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ss_item_sk] #10 + WholeStageCodegen (14) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #11 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 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 57bce0eead..10ba5439e7 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,82 +1,74 @@ == Physical Plan == -TakeOrderedAndProject (78) -+- * HashAggregate (77) - +- * ColumnarToRow (76) - +- CometColumnarExchange (75) - +- RowToColumnar (74) - +- * HashAggregate (73) - +- * Expand (72) - +- Union (71) - :- * HashAggregate (22) - : +- * ColumnarToRow (21) - : +- CometColumnarExchange (20) - : +- RowToColumnar (19) - : +- * HashAggregate (18) - : +- * Project (17) - : +- * BroadcastHashJoin Inner BuildRight (16) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (8) - : : : +- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : +- ReusedExchange (9) - : +- BroadcastExchange (15) - : +- * ColumnarToRow (14) - : +- CometFilter (13) - : +- CometScan parquet spark_catalog.default.store (12) - :- * HashAggregate (44) - : +- * ColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- RowToColumnar (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * ColumnarToRow (30) - : : : +- CometUnion (29) - : : : :- CometProject (25) - : : : : +- CometFilter (24) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (23) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) - : : +- ReusedExchange (31) - : +- BroadcastExchange (37) - : +- * ColumnarToRow (36) - : +- CometFilter (35) - : +- CometScan parquet spark_catalog.default.catalog_page (34) - +- * HashAggregate (70) - +- * ColumnarToRow (69) - +- CometColumnarExchange (68) - +- RowToColumnar (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * ColumnarToRow (56) - : : +- CometUnion (55) - : : :- CometProject (47) - : : : +- CometFilter (46) - : : : +- CometScan parquet spark_catalog.default.web_sales (45) - : : +- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastExchange (49) - : : : +- CometScan parquet spark_catalog.default.web_returns (48) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan parquet spark_catalog.default.web_sales (50) - : +- ReusedExchange (57) - +- BroadcastExchange (63) - +- * ColumnarToRow (62) - +- CometFilter (61) - +- CometScan parquet spark_catalog.default.web_site (60) +TakeOrderedAndProject (70) ++- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * Expand (66) + +- Union (65) + :- * HashAggregate (20) + : +- Exchange (19) + : +- * HashAggregate (18) + : +- * Project (17) + : +- * BroadcastHashJoin Inner BuildRight (16) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (8) + : : : +- CometUnion (7) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : +- ReusedExchange (9) + : +- BroadcastExchange (15) + : +- * ColumnarToRow (14) + : +- CometFilter (13) + : +- CometScan parquet spark_catalog.default.store (12) + :- * HashAggregate (40) + : +- Exchange (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * ColumnarToRow (28) + : : : +- CometUnion (27) + : : : :- CometProject (23) + : : : : +- CometFilter (22) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (24) + : : +- ReusedExchange (29) + : +- BroadcastExchange (35) + : +- * ColumnarToRow (34) + : +- CometFilter (33) + : +- CometScan parquet spark_catalog.default.catalog_page (32) + +- * HashAggregate (64) + +- Exchange (63) + +- * HashAggregate (62) + +- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * Project (55) + : +- * BroadcastHashJoin Inner BuildRight (54) + : :- * ColumnarToRow (52) + : : +- CometUnion (51) + : : :- CometProject (43) + : : : +- CometFilter (42) + : : : +- CometScan parquet spark_catalog.default.web_sales (41) + : : +- CometProject (50) + : : +- CometBroadcastHashJoin (49) + : : :- CometBroadcastExchange (45) + : : : +- CometScan parquet spark_catalog.default.web_returns (44) + : : +- CometProject (48) + : : +- CometFilter (47) + : : +- CometScan parquet spark_catalog.default.web_sales (46) + : +- ReusedExchange (53) + +- BroadcastExchange (59) + +- * ColumnarToRow (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.web_site (56) (unknown) Scan parquet spark_catalog.default.store_sales @@ -118,7 +110,7 @@ Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_a (8) ColumnarToRow [codegen id : 3] Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -(9) ReusedExchange [Reuses operator id: 83] +(9) ReusedExchange [Reuses operator id: 75] Output [1]: [d_date_sk#22] (10) BroadcastHashJoin [codegen id : 3] @@ -166,17 +158,11 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -(19) RowToColumnar +(19) Exchange Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(20) CometColumnarExchange -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(21) ColumnarToRow [codegen id : 4] -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] - -(22) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 4] Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] Keys [1]: [s_store_id#24] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] @@ -191,11 +177,11 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(24) CometFilter +(22) CometFilter Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Condition : isnotnull(cs_catalog_page_sk#42) -(25) CometProject +(23) CometProject Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Arguments: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52], [cs_catalog_page_sk#42 AS page_sk#47, cs_sold_date_sk#45 AS date_sk#48, cs_ext_sales_price#43 AS sales_price#49, cs_net_profit#44 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] @@ -207,31 +193,31 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(27) CometFilter +(25) CometFilter Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Condition : isnotnull(cr_catalog_page_sk#53) -(28) CometProject +(26) CometProject Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Arguments: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62], [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] -(29) CometUnion +(27) CometUnion Child 0 Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62] -(30) ColumnarToRow [codegen id : 7] +(28) ColumnarToRow [codegen id : 7] Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] -(31) ReusedExchange [Reuses operator id: 83] +(29) ReusedExchange [Reuses operator id: 75] Output [1]: [d_date_sk#63] -(32) BroadcastHashJoin [codegen id : 7] +(30) BroadcastHashJoin [codegen id : 7] Left keys [1]: [date_sk#48] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(31) Project [codegen id : 7] Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] @@ -242,45 +228,39 @@ Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter +(33) CometFilter Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Condition : isnotnull(cp_catalog_page_sk#64) -(36) ColumnarToRow [codegen id : 6] +(34) ColumnarToRow [codegen id : 6] Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -(37) BroadcastExchange +(35) BroadcastExchange Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 7] +(36) BroadcastHashJoin [codegen id : 7] Left keys [1]: [page_sk#47] Right keys [1]: [cp_catalog_page_sk#64] Join type: Inner Join condition: None -(39) Project [codegen id : 7] +(37) Project [codegen id : 7] Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#65] -(40) HashAggregate [codegen id : 7] +(38) HashAggregate [codegen id : 7] Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] Aggregate Attributes [4]: [sum#66, sum#67, sum#68, sum#69] Results [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -(41) RowToColumnar -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] - -(42) CometColumnarExchange -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(43) ColumnarToRow [codegen id : 8] +(39) Exchange Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(44) HashAggregate [codegen id : 8] +(40) HashAggregate [codegen id : 8] Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -295,11 +275,11 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(46) CometFilter +(42) CometFilter Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Condition : isnotnull(ws_web_site_sk#83) -(47) CometProject +(43) CometProject Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] @@ -310,7 +290,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] ReadSchema: struct -(49) CometBroadcastExchange +(45) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] @@ -321,40 +301,40 @@ 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 -(51) CometFilter +(47) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(52) CometProject +(48) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(53) CometBroadcastHashJoin +(49) CometBroadcastHashJoin Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(54) CometProject +(50) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -(55) CometUnion +(51) CometUnion Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] -(56) ColumnarToRow [codegen id : 11] +(52) ColumnarToRow [codegen id : 11] Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(57) ReusedExchange [Reuses operator id: 83] +(53) ReusedExchange [Reuses operator id: 75] Output [1]: [d_date_sk#109] -(58) BroadcastHashJoin [codegen id : 11] +(54) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(59) Project [codegen id : 11] +(55) Project [codegen id : 11] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -365,93 +345,81 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(61) CometFilter +(57) CometFilter Input [2]: [web_site_sk#110, web_site_id#111] Condition : isnotnull(web_site_sk#110) -(62) ColumnarToRow [codegen id : 10] +(58) ColumnarToRow [codegen id : 10] Input [2]: [web_site_sk#110, web_site_id#111] -(63) BroadcastExchange +(59) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#111] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(64) BroadcastHashJoin [codegen id : 11] +(60) BroadcastHashJoin [codegen id : 11] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(65) Project [codegen id : 11] +(61) Project [codegen id : 11] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(66) HashAggregate [codegen id : 11] +(62) HashAggregate [codegen id : 11] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -(67) RowToColumnar +(63) Exchange Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(68) CometColumnarExchange -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(69) ColumnarToRow [codegen id : 12] -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] - -(70) HashAggregate [codegen id : 12] +(64) HashAggregate [codegen id : 12] Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#126, web channel AS channel#127, concat(web_site, web_site_id#111) AS id#128] -(71) Union +(65) Union -(72) Expand [codegen id : 13] +(66) Expand [codegen id : 13] Input [5]: [sales#37, returns#38, profit#39, channel#40, id#41] Arguments: [[sales#37, returns#38, profit#39, channel#40, id#41, 0], [sales#37, returns#38, profit#39, channel#40, null, 1], [sales#37, returns#38, profit#39, null, null, 3]], [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] -(73) HashAggregate [codegen id : 13] +(67) HashAggregate [codegen id : 13] Input [6]: [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] Keys [3]: [channel#129, id#130, spark_grouping_id#131] Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] Results [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -(74) RowToColumnar -Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(75) CometColumnarExchange -Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#129, id#130, spark_grouping_id#131, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) ColumnarToRow [codegen id : 14] +(68) Exchange Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Arguments: hashpartitioning(channel#129, id#130, spark_grouping_id#131, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(77) HashAggregate [codegen id : 14] +(69) HashAggregate [codegen id : 14] Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Keys [3]: [channel#129, id#130, spark_grouping_id#131] Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] Aggregate Attributes [3]: [sum(sales#37)#144, sum(returns#38)#145, sum(profit#39)#146] Results [5]: [channel#129, id#130, sum(sales#37)#144 AS sales#147, sum(returns#38)#145 AS returns#148, sum(profit#39)#146 AS profit#149] -(78) TakeOrderedAndProject +(70) TakeOrderedAndProject Input [5]: [channel#129, id#130, sales#147, returns#148, profit#149] Arguments: 100, [channel#129 ASC NULLS FIRST, id#130 ASC NULLS FIRST], [channel#129, id#130, sales#147, returns#148, profit#149] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (83) -+- * ColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan parquet spark_catalog.default.date_dim (79) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) (unknown) Scan parquet spark_catalog.default.date_dim @@ -461,29 +429,29 @@ 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 -(80) CometFilter +(72) CometFilter Input [2]: [d_date_sk#22, d_date#150] Condition : (((isnotnull(d_date#150) AND (d_date#150 >= 2000-08-23)) AND (d_date#150 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(81) CometProject +(73) CometProject Input [2]: [d_date_sk#22, d_date#150] Arguments: [d_date_sk#22], [d_date_sk#22] -(82) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(83) BroadcastExchange +(75) 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 = 23 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 48 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 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 689f242fb1..3d539d5917 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,119 +1,111 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (14) 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - RowToColumnar - WholeStageCodegen (13) - 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 (4) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan 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] - CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + Exchange [channel,id,spark_grouping_id] #1 + WholeStageCodegen (13) + 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 (4) + 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] + InputAdapter + Exchange [s_store_id] #2 + WholeStageCodegen (3) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan 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] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (8) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #5 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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] - CometScan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) + 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] + InputAdapter + Exchange [cp_catalog_page_id] #5 + WholeStageCodegen (7) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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] + CometScan 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 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (12) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #7 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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,ws_item_sk,ws_order_number] - CometBroadcastExchange #8 - CometScan 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_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (12) + 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] + InputAdapter + Exchange [web_site_id] #7 + WholeStageCodegen (11) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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,ws_item_sk,ws_order_number] + CometBroadcastExchange #8 + CometScan 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_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (10) + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + CometFilter [web_site_sk] + CometScan 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/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt index ea1267cb3a..b178faed1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt @@ -1,34 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- * ColumnarToRow (28) - +- CometColumnarExchange (27) - +- RowToColumnar (26) - +- * HashAggregate (25) - +- * Project (24) - +- * BroadcastHashJoin Inner BuildRight (23) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.date_dim (16) - +- ReusedExchange (22) +TakeOrderedAndProject (28) ++- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * Project (24) + +- * BroadcastHashJoin Inner BuildRight (23) + :- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store (10) + : +- BroadcastExchange (19) + : +- * ColumnarToRow (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.date_dim (16) + +- ReusedExchange (22) (unknown) Scan parquet spark_catalog.default.store_sales @@ -131,7 +129,7 @@ Join condition: None Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#22] -(22) ReusedExchange [Reuses operator id: 35] +(22) ReusedExchange [Reuses operator id: 33] Output [1]: [d_date_sk#23] (23) BroadcastHashJoin [codegen id : 5] @@ -151,35 +149,29 @@ Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk# Aggregate Attributes [5]: [sum#24, sum#25, sum#26, sum#27, sum#28] Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] -(26) RowToColumnar +(26) Exchange Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(27) CometColumnarExchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(28) ColumnarToRow [codegen id : 6] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] - -(29) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 6] Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] Aggregate Attributes [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38] Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34 AS 30 days #39, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35 AS 31 - 60 days #40, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36 AS 61 - 90 days #41, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37 AS 91 - 120 days #42, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38 AS >120 days #43] -(30) TakeOrderedAndProject +(28) TakeOrderedAndProject Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) (unknown) Scan parquet spark_catalog.default.date_dim @@ -189,18 +181,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter +(30) CometFilter Input [3]: [d_date_sk#23, d_year#44, d_moy#45] Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 8)) AND isnotnull(d_date_sk#23)) -(33) CometProject +(31) CometProject Input [3]: [d_date_sk#23, d_year#44, d_moy#45] Arguments: [d_date_sk#23], [d_date_sk#23] -(34) ColumnarToRow [codegen id : 1] +(32) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#23] -(35) BroadcastExchange +(33) BroadcastExchange Input [1]: [d_date_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt index 0d5c346cf5..dfdcaf4975 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt @@ -1,52 +1,50 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] WholeStageCodegen (6) HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + InputAdapter + Exchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 + WholeStageCodegen (5) + HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt index 5b392331bf..c55b122617 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt @@ -1,52 +1,41 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Filter (47) - +- Window (46) - +- * ColumnarToRow (45) - +- CometSort (44) - +- CometColumnarExchange (43) - +- CometProject (42) - +- CometSortMergeJoin (41) - :- CometSort (20) - : +- CometColumnarExchange (19) - : +- RowToColumnar (18) - : +- * Project (17) - : +- Window (16) - : +- * ColumnarToRow (15) - : +- CometSort (14) - : +- CometColumnarExchange (13) - : +- RowToColumnar (12) - : +- * HashAggregate (11) - : +- * ColumnarToRow (10) - : +- CometColumnarExchange (9) - : +- RowToColumnar (8) - : +- * HashAggregate (7) - : +- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- CometSort (40) - +- CometColumnarExchange (39) - +- RowToColumnar (38) - +- * Project (37) - +- Window (36) - +- * ColumnarToRow (35) - +- CometSort (34) - +- CometColumnarExchange (33) - +- RowToColumnar (32) - +- * HashAggregate (31) - +- * ColumnarToRow (30) - +- CometColumnarExchange (29) - +- RowToColumnar (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * ColumnarToRow (23) - : +- CometFilter (22) - : +- CometScan parquet spark_catalog.default.store_sales (21) - +- ReusedExchange (24) +TakeOrderedAndProject (37) ++- * Filter (36) + +- Window (35) + +- * Sort (34) + +- Exchange (33) + +- * Project (32) + +- * SortMergeJoin FullOuter (31) + :- * Sort (15) + : +- Exchange (14) + : +- * Project (13) + : +- Window (12) + : +- * Sort (11) + : +- Exchange (10) + : +- * HashAggregate (9) + : +- Exchange (8) + : +- * HashAggregate (7) + : +- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- ReusedExchange (4) + +- * Sort (30) + +- Exchange (29) + +- * Project (28) + +- Window (27) + +- * Sort (26) + +- Exchange (25) + +- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildRight (20) + :- * ColumnarToRow (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.store_sales (16) + +- ReusedExchange (19) (unknown) Scan parquet spark_catalog.default.web_sales @@ -64,7 +53,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 2] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 53] +(4) ReusedExchange [Reuses operator id: 42] Output [2]: [d_date_sk#5, d_date#6] (5) BroadcastHashJoin [codegen id : 2] @@ -84,55 +73,40 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] Aggregate Attributes [1]: [sum#7] Results [3]: [ws_item_sk#1, d_date#6, sum#8] -(8) RowToColumnar +(8) Exchange Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) CometColumnarExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(10) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(11) HashAggregate [codegen id : 3] +(9) HashAggregate [codegen id : 3] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS _w0#11, ws_item_sk#1] -(12) RowToColumnar -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] - -(13) CometColumnarExchange +(10) Exchange Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(14) CometSort +(11) Sort [codegen id : 4] Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(15) ColumnarToRow [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] - -(16) Window +(12) Window Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(17) Project [codegen id : 5] +(13) Project [codegen id : 5] Output [3]: [item_sk#10, d_date#6, cume_sales#12] Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] -(18) RowToColumnar -Input [3]: [item_sk#10, d_date#6, cume_sales#12] - -(19) CometColumnarExchange +(14) Exchange Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) CometSort +(15) Sort [codegen id : 6] Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +Arguments: [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] @@ -142,123 +116,106 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(22) CometFilter +(17) CometFilter Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Condition : isnotnull(ss_item_sk#13) -(23) ColumnarToRow [codegen id : 7] +(18) ColumnarToRow [codegen id : 8] Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -(24) ReusedExchange [Reuses operator id: 53] +(19) ReusedExchange [Reuses operator id: 42] Output [2]: [d_date_sk#17, d_date#18] -(25) BroadcastHashJoin [codegen id : 7] +(20) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_sold_date_sk#15] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(26) Project [codegen id : 7] +(21) Project [codegen id : 8] Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] -(27) HashAggregate [codegen id : 7] +(22) HashAggregate [codegen id : 8] Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] Aggregate Attributes [1]: [sum#19] Results [3]: [ss_item_sk#13, d_date#18, sum#20] -(28) RowToColumnar -Input [3]: [ss_item_sk#13, d_date#18, sum#20] - -(29) CometColumnarExchange +(23) Exchange Input [3]: [ss_item_sk#13, d_date#18, sum#20] -Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(30) ColumnarToRow [codegen id : 8] -Input [3]: [ss_item_sk#13, d_date#18, sum#20] - -(31) HashAggregate [codegen id : 8] +(24) HashAggregate [codegen id : 9] Input [3]: [ss_item_sk#13, d_date#18, sum#20] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#21] Results [4]: [ss_item_sk#13 AS item_sk#22, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#21,17,2) AS _w0#23, ss_item_sk#13] -(32) RowToColumnar -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] - -(33) CometColumnarExchange -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort +(25) Exchange Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -Arguments: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] +Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(35) ColumnarToRow [codegen id : 9] +(26) Sort [codegen id : 10] Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] +Arguments: [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 -(36) Window +(27) Window Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] Arguments: [sum(_w0#23) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#24], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] -(37) Project [codegen id : 10] +(28) Project [codegen id : 11] Output [3]: [item_sk#22, d_date#18, cume_sales#24] Input [5]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13, cume_sales#24] -(38) RowToColumnar +(29) Exchange Input [3]: [item_sk#22, d_date#18, cume_sales#24] +Arguments: hashpartitioning(item_sk#22, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(39) CometColumnarExchange +(30) Sort [codegen id : 12] Input [3]: [item_sk#22, d_date#18, cume_sales#24] -Arguments: hashpartitioning(item_sk#22, d_date#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: [item_sk#22 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 -(40) CometSort -Input [3]: [item_sk#22, d_date#18, cume_sales#24] -Arguments: [item_sk#22, d_date#18, cume_sales#24], [item_sk#22 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] - -(41) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#12] -Right output [3]: [item_sk#22, d_date#18, cume_sales#24] -Arguments: [item_sk#10, d_date#6], [item_sk#22, d_date#18], FullOuter +(31) SortMergeJoin [codegen id : 13] +Left keys [2]: [item_sk#10, d_date#6] +Right keys [2]: [item_sk#22, d_date#18] +Join type: FullOuter +Join condition: None -(42) CometProject +(32) Project [codegen id : 13] +Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#22 END AS item_sk#25, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#26, cume_sales#12 AS web_sales#27, cume_sales#24 AS store_sales#28] Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#22, d_date#18, cume_sales#24] -Arguments: [item_sk#25, d_date#26, web_sales#27, store_sales#28], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#22 END AS item_sk#25, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#26, cume_sales#12 AS web_sales#27, cume_sales#24 AS store_sales#28] - -(43) CometColumnarExchange -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: hashpartitioning(item_sk#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(44) CometSort +(33) Exchange Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: [item_sk#25, d_date#26, web_sales#27, store_sales#28], [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST] +Arguments: hashpartitioning(item_sk#25, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(45) ColumnarToRow [codegen id : 11] +(34) Sort [codegen id : 14] Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] +Arguments: [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], false, 0 -(46) Window +(35) Window Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] Arguments: [max(web_sales#27) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#29, max(store_sales#28) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#30], [item_sk#25], [d_date#26 ASC NULLS FIRST] -(47) Filter [codegen id : 12] +(36) Filter [codegen id : 15] Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] Condition : ((isnotnull(web_cumulative#29) AND isnotnull(store_cumulative#30)) AND (web_cumulative#29 > store_cumulative#30)) -(48) TakeOrderedAndProject +(37) TakeOrderedAndProject Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] Arguments: 100, [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (42) ++- * ColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.date_dim (38) (unknown) Scan parquet spark_catalog.default.date_dim @@ -268,21 +225,21 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(39) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1200)) AND (d_month_seq#31 <= 1211)) AND isnotnull(d_date_sk#5)) -(51) CometProject +(40) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(52) ColumnarToRow [codegen id : 1] +(41) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(53) BroadcastExchange +(42) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index 4814b3ce60..181cd1b98c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -1,79 +1,75 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (12) + WholeStageCodegen (15) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (11) - ColumnarToRow + WholeStageCodegen (14) + Sort [item_sk,d_date] InputAdapter - CometSort [item_sk,d_date] - CometColumnarExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,item_sk,d_date] - CometSort [item_sk,d_date] - CometColumnarExchange [item_sk,d_date] #2 - RowToColumnar - WholeStageCodegen (5) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometSort [ws_item_sk,d_date] - CometColumnarExchange [ws_item_sk] #3 - RowToColumnar + Exchange [item_sk] #1 + WholeStageCodegen (13) + Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] + SortMergeJoin [item_sk,d_date,item_sk,d_date] + InputAdapter + WholeStageCodegen (6) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #2 + WholeStageCodegen (5) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (4) + Sort [ws_item_sk,d_date] + InputAdapter + Exchange [ws_item_sk] #3 WholeStageCodegen (3) HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #4 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - CometSort [item_sk,d_date] - CometColumnarExchange [item_sk,d_date] #6 - RowToColumnar - WholeStageCodegen (10) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometSort [ss_item_sk,d_date] - CometColumnarExchange [ss_item_sk] #7 - RowToColumnar - WholeStageCodegen (8) + InputAdapter + Exchange [ws_item_sk,d_date] #4 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (12) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #6 + WholeStageCodegen (11) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (10) + Sort [ss_item_sk,d_date] + InputAdapter + Exchange [ss_item_sk] #7 + WholeStageCodegen (9) HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #8 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + Exchange [ss_item_sk,d_date] #8 + WholeStageCodegen (8) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt index d95223d318..8df157e51b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt @@ -1,27 +1,25 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- * ColumnarToRow (21) - +- CometColumnarExchange (20) - +- RowToColumnar (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- Exchange (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) (unknown) Scan parquet spark_catalog.default.date_dim @@ -110,24 +108,18 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -(19) RowToColumnar +(19) Exchange Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) CometColumnarExchange -Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) ColumnarToRow [codegen id : 4] -Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] - -(22) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 4] Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(23) TakeOrderedAndProject +(21) TakeOrderedAndProject Input [4]: [d_year#2, brand_id#14, brand#15, ext_price#16] Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, ext_price#16] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt index 8d8488b9fb..91fdc2f17d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt @@ -1,33 +1,31 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (4) HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [d_year,i_brand,i_brand_id] #1 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + Exchange [d_year,i_brand,i_brand_id] #1 + WholeStageCodegen (3) + HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 251b83b596..5142c9e4e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -1,36 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * Filter (30) - +- Window (29) - +- * ColumnarToRow (28) - +- CometSort (27) - +- CometColumnarExchange (26) - +- RowToColumnar (25) - +- * HashAggregate (24) - +- * ColumnarToRow (23) - +- CometColumnarExchange (22) - +- RowToColumnar (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.store (14) +TakeOrderedAndProject (28) ++- * Project (27) + +- * Filter (26) + +- Window (25) + +- * Sort (24) + +- Exchange (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.store_sales (5) + : +- ReusedExchange (11) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) (unknown) Scan parquet spark_catalog.default.item @@ -80,7 +76,7 @@ Join condition: None Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -(11) ReusedExchange [Reuses operator id: 37] +(11) ReusedExchange [Reuses operator id: 33] Output [2]: [d_date_sk#15, d_qoy#16] (12) BroadcastHashJoin [codegen id : 4] @@ -128,61 +124,49 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] Aggregate Attributes [1]: [sum#18] Results [3]: [i_manufact_id#5, d_qoy#16, sum#19] -(21) RowToColumnar +(21) Exchange Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] +Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) CometColumnarExchange -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) ColumnarToRow [codegen id : 5] -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] - -(24) HashAggregate [codegen id : 5] +(22) HashAggregate [codegen id : 5] Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] Keys [2]: [i_manufact_id#5, d_qoy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] -(25) RowToColumnar -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] - -(26) CometColumnarExchange -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort +(23) Exchange Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] +Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) ColumnarToRow [codegen id : 6] +(24) Sort [codegen id : 6] Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +Arguments: [i_manufact_id#5 ASC NULLS FIRST], false, 0 -(29) Window +(25) Window Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] -(30) Filter [codegen id : 7] +(26) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END -(31) Project [codegen id : 7] +(27) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -(32) TakeOrderedAndProject +(28) TakeOrderedAndProject Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) (unknown) Scan parquet spark_catalog.default.date_dim @@ -192,18 +176,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(30) CometFilter Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(35) CometProject +(31) CometProject Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] -(36) ColumnarToRow [codegen id : 1] +(32) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#16] -(37) BroadcastExchange +(33) BroadcastExchange Input [2]: [d_date_sk#15, d_qoy#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index c0908b5397..adda5c34f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -5,51 +5,47 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] InputAdapter Window [_w0,i_manufact_id] WholeStageCodegen (6) - ColumnarToRow + Sort [i_manufact_id] InputAdapter - CometSort [i_manufact_id] - CometColumnarExchange [i_manufact_id] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id,d_qoy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - Project [i_manufact_id,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_qoy] #4 + Exchange [i_manufact_id] #1 + WholeStageCodegen (5) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_manufact_id,d_qoy] #2 + WholeStageCodegen (4) + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + Project [i_manufact_id,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_category,i_class,i_brand,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_qoy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 046229f1e8..5c1e750c6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -1,65 +1,59 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * HashAggregate (60) - +- * ColumnarToRow (59) - +- CometColumnarExchange (58) - +- RowToColumnar (57) - +- * HashAggregate (56) - +- * HashAggregate (55) - +- * ColumnarToRow (54) - +- CometColumnarExchange (53) - +- RowToColumnar (52) - +- * HashAggregate (51) - +- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (47) - : +- * BroadcastHashJoin Inner BuildRight (46) - : :- * Project (41) - : : +- * BroadcastHashJoin Inner BuildRight (40) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (29) - : : : : +- CometHashAggregate (28) - : : : : +- CometColumnarExchange (27) - : : : : +- RowToColumnar (26) - : : : : +- * HashAggregate (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * ColumnarToRow (8) - : : : : : : : +- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (4) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- CometProject (11) - : : : : : : +- CometFilter (10) - : : : : : : +- CometScan parquet spark_catalog.default.item (9) - : : : : : +- ReusedExchange (16) - : : : : +- BroadcastExchange (22) - : : : : +- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (33) - : : : +- * ColumnarToRow (32) - : : : +- CometFilter (31) - : : : +- CometScan parquet spark_catalog.default.store_sales (30) - : : +- BroadcastExchange (39) - : : +- * ColumnarToRow (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.customer_address (36) - : +- BroadcastExchange (45) - : +- * ColumnarToRow (44) - : +- CometFilter (43) - : +- CometScan parquet spark_catalog.default.store (42) - +- ReusedExchange (48) +TakeOrderedAndProject (55) ++- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * HashAggregate (51) + +- Exchange (50) + +- * HashAggregate (49) + +- * Project (48) + +- * BroadcastHashJoin Inner BuildRight (47) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * Project (33) + : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : :- * HashAggregate (27) + : : : : +- Exchange (26) + : : : : +- * HashAggregate (25) + : : : : +- * Project (24) + : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * ColumnarToRow (8) + : : : : : : : +- CometUnion (7) + : : : : : : : :- CometProject (3) + : : : : : : : : +- CometFilter (2) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : +- CometProject (6) + : : : : : : : +- CometFilter (5) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (4) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * ColumnarToRow (12) + : : : : : : +- CometProject (11) + : : : : : : +- CometFilter (10) + : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : +- ReusedExchange (16) + : : : : +- BroadcastExchange (22) + : : : : +- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (31) + : : : +- * ColumnarToRow (30) + : : : +- CometFilter (29) + : : : +- CometScan parquet spark_catalog.default.store_sales (28) + : : +- BroadcastExchange (37) + : : +- * ColumnarToRow (36) + : : +- CometFilter (35) + : : +- CometScan parquet spark_catalog.default.customer_address (34) + : +- BroadcastExchange (43) + : +- * ColumnarToRow (42) + : +- CometFilter (41) + : +- CometScan parquet spark_catalog.default.store (40) + +- ReusedExchange (46) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -133,7 +127,7 @@ Join condition: None Output [2]: [sold_date_sk#5, customer_sk#6] Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] -(16) ReusedExchange [Reuses operator id: 66] +(16) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#17] (17) BroadcastHashJoin [codegen id : 4] @@ -181,20 +175,16 @@ Functions: [] Aggregate Attributes: [] Results [2]: [c_customer_sk#18, c_current_addr_sk#19] -(26) RowToColumnar +(26) Exchange Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +Arguments: hashpartitioning(c_customer_sk#18, c_current_addr_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(27) CometColumnarExchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: hashpartitioning(c_customer_sk#18, c_current_addr_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometHashAggregate +(27) HashAggregate [codegen id : 9] Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] Functions: [] - -(29) ColumnarToRow [codegen id : 9] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +Aggregate Attributes: [] +Results [2]: [c_customer_sk#18, c_current_addr_sk#19] (unknown) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] @@ -204,24 +194,24 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(31) CometFilter +(29) CometFilter Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) -(32) ColumnarToRow [codegen id : 5] +(30) ColumnarToRow [codegen id : 5] Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -(33) BroadcastExchange +(31) BroadcastExchange Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#18] Right keys [1]: [ss_customer_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 9] +(33) Project [codegen id : 9] Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] @@ -232,24 +222,24 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county), IsNotNull(ca_state)] ReadSchema: struct -(37) CometFilter +(35) CometFilter Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnotnull(ca_state#26)) -(38) ColumnarToRow [codegen id : 6] +(36) ColumnarToRow [codegen id : 6] Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -(39) BroadcastExchange +(37) BroadcastExchange Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(40) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#19] Right keys [1]: [ca_address_sk#24] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(39) Project [codegen id : 9] Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26] Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#26] @@ -260,100 +250,88 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] ReadSchema: struct -(43) CometFilter +(41) CometFilter Input [2]: [s_county#27, s_state#28] Condition : (isnotnull(s_county#27) AND isnotnull(s_state#28)) -(44) ColumnarToRow [codegen id : 7] +(42) ColumnarToRow [codegen id : 7] Input [2]: [s_county#27, s_state#28] -(45) BroadcastExchange +(43) BroadcastExchange Input [2]: [s_county#27, s_state#28] Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 9] Left keys [2]: [ca_county#25, ca_state#26] Right keys [2]: [s_county#27, s_state#28] Join type: Inner Join condition: None -(47) Project [codegen id : 9] +(45) Project [codegen id : 9] Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26, s_county#27, s_state#28] -(48) ReusedExchange [Reuses operator id: 71] +(46) ReusedExchange [Reuses operator id: 65] Output [1]: [d_date_sk#29] -(49) BroadcastHashJoin [codegen id : 9] +(47) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_sold_date_sk#22] Right keys [1]: [d_date_sk#29] Join type: Inner Join condition: None -(50) Project [codegen id : 9] +(48) Project [codegen id : 9] Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#29] -(51) HashAggregate [codegen id : 9] +(49) HashAggregate [codegen id : 9] Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] Keys [1]: [c_customer_sk#18] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] Aggregate Attributes [1]: [sum#30] Results [2]: [c_customer_sk#18, sum#31] -(52) RowToColumnar +(50) Exchange Input [2]: [c_customer_sk#18, sum#31] +Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(53) CometColumnarExchange -Input [2]: [c_customer_sk#18, sum#31] -Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(54) ColumnarToRow [codegen id : 10] -Input [2]: [c_customer_sk#18, sum#31] - -(55) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 10] Input [2]: [c_customer_sk#18, sum#31] Keys [1]: [c_customer_sk#18] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#32] Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#32,17,2) / 50) as int) AS segment#33] -(56) HashAggregate [codegen id : 10] +(52) HashAggregate [codegen id : 10] Input [1]: [segment#33] Keys [1]: [segment#33] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#34] Results [2]: [segment#33, count#35] -(57) RowToColumnar -Input [2]: [segment#33, count#35] - -(58) CometColumnarExchange -Input [2]: [segment#33, count#35] -Arguments: hashpartitioning(segment#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(59) ColumnarToRow [codegen id : 11] +(53) Exchange Input [2]: [segment#33, count#35] +Arguments: hashpartitioning(segment#33, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(60) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 11] Input [2]: [segment#33, count#35] Keys [1]: [segment#33] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#36] Results [3]: [segment#33, count(1)#36 AS num_customers#37, (segment#33 * 50) AS segment_base#38] -(61) TakeOrderedAndProject +(55) TakeOrderedAndProject Input [3]: [segment#33, num_customers#37, segment_base#38] Arguments: 100, [segment#33 ASC NULLS FIRST, num_customers#37 ASC NULLS FIRST], [segment#33, num_customers#37, segment_base#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (66) -+- * ColumnarToRow (65) - +- CometProject (64) - +- CometFilter (63) - +- CometScan parquet spark_catalog.default.date_dim (62) +BroadcastExchange (60) ++- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) (unknown) Scan parquet spark_catalog.default.date_dim @@ -363,29 +341,29 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(63) CometFilter +(57) CometFilter Input [3]: [d_date_sk#17, d_year#39, d_moy#40] Condition : ((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 = 12)) AND (d_year#39 = 1998)) AND isnotnull(d_date_sk#17)) -(64) CometProject +(58) CometProject Input [3]: [d_date_sk#17, d_year#39, d_moy#40] Arguments: [d_date_sk#17], [d_date_sk#17] -(65) ColumnarToRow [codegen id : 1] +(59) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] -(66) BroadcastExchange +(60) BroadcastExchange Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (71) -+- * ColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan parquet spark_catalog.default.date_dim (67) +Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (65) ++- * ColumnarToRow (64) + +- CometProject (63) + +- CometFilter (62) + +- CometScan parquet spark_catalog.default.date_dim (61) (unknown) Scan parquet spark_catalog.default.date_dim @@ -395,29 +373,29 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(68) CometFilter +(62) CometFilter Input [2]: [d_date_sk#29, d_month_seq#41] Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= Subquery scalar-subquery#42, [id=#43])) AND (d_month_seq#41 <= Subquery scalar-subquery#44, [id=#45])) AND isnotnull(d_date_sk#29)) -(69) CometProject +(63) CometProject Input [2]: [d_date_sk#29, d_month_seq#41] Arguments: [d_date_sk#29], [d_date_sk#29] -(70) ColumnarToRow [codegen id : 1] +(64) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#29] -(71) BroadcastExchange +(65) BroadcastExchange Input [1]: [d_date_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:4 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* ColumnarToRow (78) -+- CometHashAggregate (77) - +- CometColumnarExchange (76) - +- CometHashAggregate (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +Subquery:4 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#42, [id=#43] +* HashAggregate (72) ++- Exchange (71) + +- * ColumnarToRow (70) + +- CometHashAggregate (69) + +- CometProject (68) + +- CometFilter (67) + +- CometScan parquet spark_catalog.default.date_dim (66) (unknown) Scan parquet spark_catalog.default.date_dim @@ -427,39 +405,41 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(73) CometFilter +(67) CometFilter Input [3]: [d_month_seq#46, d_year#47, d_moy#48] Condition : (((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 12)) -(74) CometProject +(68) CometProject Input [3]: [d_month_seq#46, d_year#47, d_moy#48] Arguments: [(d_month_seq + 1)#49], [(d_month_seq#46 + 1) AS (d_month_seq + 1)#49] -(75) CometHashAggregate +(69) CometHashAggregate Input [1]: [(d_month_seq + 1)#49] Keys [1]: [(d_month_seq + 1)#49] Functions: [] -(76) CometColumnarExchange +(70) ColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 1)#49] -Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(77) CometHashAggregate +(71) Exchange Input [1]: [(d_month_seq + 1)#49] -Keys [1]: [(d_month_seq + 1)#49] -Functions: [] +Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(78) ColumnarToRow [codegen id : 1] +(72) HashAggregate [codegen id : 2] Input [1]: [(d_month_seq + 1)#49] +Keys [1]: [(d_month_seq + 1)#49] +Functions: [] +Aggregate Attributes: [] +Results [1]: [(d_month_seq + 1)#49] -Subquery:5 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* ColumnarToRow (85) -+- CometHashAggregate (84) - +- CometColumnarExchange (83) - +- CometHashAggregate (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan parquet spark_catalog.default.date_dim (79) +Subquery:5 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* HashAggregate (79) ++- Exchange (78) + +- * ColumnarToRow (77) + +- CometHashAggregate (76) + +- CometProject (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) (unknown) Scan parquet spark_catalog.default.date_dim @@ -469,29 +449,31 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(80) CometFilter +(74) CometFilter Input [3]: [d_month_seq#50, d_year#51, d_moy#52] Condition : (((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 1998)) AND (d_moy#52 = 12)) -(81) CometProject +(75) CometProject Input [3]: [d_month_seq#50, d_year#51, d_moy#52] Arguments: [(d_month_seq + 3)#53], [(d_month_seq#50 + 3) AS (d_month_seq + 3)#53] -(82) CometHashAggregate +(76) CometHashAggregate Input [1]: [(d_month_seq + 3)#53] Keys [1]: [(d_month_seq + 3)#53] Functions: [] -(83) CometColumnarExchange +(77) ColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 3)#53] -Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(84) CometHashAggregate +(78) Exchange Input [1]: [(d_month_seq + 3)#53] -Keys [1]: [(d_month_seq + 3)#53] -Functions: [] +Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(85) ColumnarToRow [codegen id : 1] +(79) HashAggregate [codegen id : 2] Input [1]: [(d_month_seq + 3)#53] +Keys [1]: [(d_month_seq + 3)#53] +Functions: [] +Aggregate Attributes: [] +Results [1]: [(d_month_seq + 3)#53] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index 43ff7aad8f..c1690facd4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -1,123 +1,121 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (11) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [segment] #1 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #2 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] - Project [c_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - BroadcastHashJoin [customer_sk,c_customer_sk] - Project [customer_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometUnion - CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #5 + InputAdapter + Exchange [segment] #1 + WholeStageCodegen (10) + HashAggregate [segment] [count,count] + HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] + InputAdapter + Exchange [c_customer_sk] #2 + WholeStageCodegen (9) + HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] + Project [c_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ca_county,ca_state,s_county,s_state] + Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,c_current_addr_sk] + InputAdapter + Exchange [c_customer_sk,c_current_addr_sk] #3 + WholeStageCodegen (4) + HashAggregate [c_customer_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk] + BroadcastHashJoin [customer_sk,c_customer_sk] + Project [customer_sk] + BroadcastHashJoin [sold_date_sk,d_date_sk] + Project [sold_date_sk,customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - Subquery #3 + CometProject [i_item_sk] + CometFilter [i_category,i_class,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + Subquery #3 + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 1)] + InputAdapter + Exchange [(d_month_seq + 1)] #9 WholeStageCodegen (1) ColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 1)] - CometColumnarExchange [(d_month_seq + 1)] #9 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 3)] + InputAdapter + Exchange [(d_month_seq + 3)] #10 WholeStageCodegen (1) ColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 3)] - CometColumnarExchange [(d_month_seq + 3)] #10 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (6) + ColumnarToRow InputAdapter - BroadcastExchange #11 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_county,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] + CometFilter [ca_address_sk,ca_county,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (7) + ColumnarToRow InputAdapter - BroadcastExchange #12 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [s_county,s_state] - CometScan parquet spark_catalog.default.store [s_county,s_state] - InputAdapter - ReusedExchange [d_date_sk] #8 + CometFilter [s_county,s_state] + CometScan parquet spark_catalog.default.store [s_county,s_state] + InputAdapter + ReusedExchange [d_date_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt index 402566a451..2efd757741 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt @@ -1,27 +1,25 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- * ColumnarToRow (21) - +- CometColumnarExchange (20) - +- RowToColumnar (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- Exchange (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) (unknown) Scan parquet spark_catalog.default.date_dim @@ -110,24 +108,18 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [3]: [i_brand#9, i_brand_id#8, sum#12] -(19) RowToColumnar +(19) Exchange Input [3]: [i_brand#9, i_brand_id#8, sum#12] +Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) CometColumnarExchange -Input [3]: [i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) ColumnarToRow [codegen id : 4] -Input [3]: [i_brand#9, i_brand_id#8, sum#12] - -(22) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 4] Input [3]: [i_brand#9, i_brand_id#8, sum#12] Keys [2]: [i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] Results [3]: [i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(23) TakeOrderedAndProject +(21) TakeOrderedAndProject Input [3]: [brand_id#14, brand#15, ext_price#16] Arguments: 100, [ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [brand_id#14, brand#15, ext_price#16] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt index 30cc73bfd4..7a0fe88633 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt @@ -1,33 +1,31 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (4) HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand,i_brand_id] #1 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + Exchange [i_brand,i_brand_id] #1 + WholeStageCodegen (3) + HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 64b27c6b38..0fb6eb172f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -1,75 +1,67 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * HashAggregate (70) - +- * ColumnarToRow (69) - +- CometColumnarExchange (68) - +- RowToColumnar (67) - +- * HashAggregate (66) - +- Union (65) - :- * HashAggregate (30) - : +- * ColumnarToRow (29) - : +- CometColumnarExchange (28) - : +- RowToColumnar (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - :- * HashAggregate (47) - : +- * ColumnarToRow (46) - : +- CometColumnarExchange (45) - : +- RowToColumnar (44) - : +- * HashAggregate (43) - : +- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (36) - : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : :- * ColumnarToRow (33) - : : : : +- CometFilter (32) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : : +- ReusedExchange (34) - : : +- ReusedExchange (37) - : +- ReusedExchange (40) - +- * HashAggregate (64) - +- * ColumnarToRow (63) - +- CometColumnarExchange (62) - +- RowToColumnar (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * ColumnarToRow (50) - : : : +- CometFilter (49) - : : : +- CometScan parquet spark_catalog.default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) +TakeOrderedAndProject (63) ++- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * ColumnarToRow (31) + : : : : +- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * ColumnarToRow (46) + : : : +- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (unknown) Scan parquet spark_catalog.default.store_sales @@ -87,7 +79,7 @@ Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 76] +(4) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -195,17 +187,11 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#13] Results [2]: [i_item_id#10, sum#14] -(27) RowToColumnar +(27) Exchange Input [2]: [i_item_id#10, sum#14] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) CometColumnarExchange -Input [2]: [i_item_id#10, sum#14] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_id#10, sum#14] - -(30) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 6] Input [2]: [i_item_id#10, sum#14] Keys [1]: [i_item_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -220,70 +206,64 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(32) CometFilter +(30) CometFilter Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(33) ColumnarToRow [codegen id : 11] +(31) ColumnarToRow [codegen id : 11] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -(34) ReusedExchange [Reuses operator id: 76] +(32) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#22] -(35) BroadcastHashJoin [codegen id : 11] +(33) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#20] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(36) Project [codegen id : 11] +(34) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -(37) ReusedExchange [Reuses operator id: 11] +(35) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#23] -(38) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#17] Right keys [1]: [ca_address_sk#23] Join type: Inner Join condition: None -(39) Project [codegen id : 11] +(37) Project [codegen id : 11] Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] -(40) ReusedExchange [Reuses operator id: 23] +(38) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#24, i_item_id#25] -(41) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#18] Right keys [1]: [i_item_sk#24] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(40) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#19, i_item_id#25] Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] -(43) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#19, i_item_id#25] Keys [1]: [i_item_id#25] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] Aggregate Attributes [1]: [sum#26] Results [2]: [i_item_id#25, sum#27] -(44) RowToColumnar -Input [2]: [i_item_id#25, sum#27] - -(45) CometColumnarExchange -Input [2]: [i_item_id#25, sum#27] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(46) ColumnarToRow [codegen id : 12] +(42) Exchange Input [2]: [i_item_id#25, sum#27] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(47) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 12] Input [2]: [i_item_id#25, sum#27] Keys [1]: [i_item_id#25] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] @@ -298,114 +278,102 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) CometFilter +(45) CometFilter Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) -(50) ColumnarToRow [codegen id : 17] +(46) ColumnarToRow [codegen id : 17] Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -(51) ReusedExchange [Reuses operator id: 76] +(47) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#35] -(52) BroadcastHashJoin [codegen id : 17] +(48) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_sold_date_sk#33] Right keys [1]: [d_date_sk#35] Join type: Inner Join condition: None -(53) Project [codegen id : 17] +(49) Project [codegen id : 17] Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] -(54) ReusedExchange [Reuses operator id: 11] +(50) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#36] -(55) BroadcastHashJoin [codegen id : 17] +(51) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_bill_addr_sk#31] Right keys [1]: [ca_address_sk#36] Join type: Inner Join condition: None -(56) Project [codegen id : 17] +(52) Project [codegen id : 17] Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -(57) ReusedExchange [Reuses operator id: 23] +(53) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#37, i_item_id#38] -(58) BroadcastHashJoin [codegen id : 17] +(54) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_item_sk#30] Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(55) Project [codegen id : 17] Output [2]: [ws_ext_sales_price#32, i_item_id#38] Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] -(60) HashAggregate [codegen id : 17] +(56) HashAggregate [codegen id : 17] Input [2]: [ws_ext_sales_price#32, i_item_id#38] Keys [1]: [i_item_id#38] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] Aggregate Attributes [1]: [sum#39] Results [2]: [i_item_id#38, sum#40] -(61) RowToColumnar +(57) Exchange Input [2]: [i_item_id#38, sum#40] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(62) CometColumnarExchange -Input [2]: [i_item_id#38, sum#40] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(63) ColumnarToRow [codegen id : 18] -Input [2]: [i_item_id#38, sum#40] - -(64) HashAggregate [codegen id : 18] +(58) HashAggregate [codegen id : 18] Input [2]: [i_item_id#38, sum#40] Keys [1]: [i_item_id#38] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] -(65) Union +(59) Union -(66) HashAggregate [codegen id : 19] +(60) HashAggregate [codegen id : 19] Input [2]: [i_item_id#10, total_sales#16] Keys [1]: [i_item_id#10] Functions [1]: [partial_sum(total_sales#16)] Aggregate Attributes [2]: [sum#43, isEmpty#44] Results [3]: [i_item_id#10, sum#45, isEmpty#46] -(67) RowToColumnar -Input [3]: [i_item_id#10, sum#45, isEmpty#46] - -(68) CometColumnarExchange -Input [3]: [i_item_id#10, sum#45, isEmpty#46] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(69) ColumnarToRow [codegen id : 20] +(61) Exchange Input [3]: [i_item_id#10, sum#45, isEmpty#46] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(70) HashAggregate [codegen id : 20] +(62) HashAggregate [codegen id : 20] Input [3]: [i_item_id#10, sum#45, isEmpty#46] Keys [1]: [i_item_id#10] Functions [1]: [sum(total_sales#16)] Aggregate Attributes [1]: [sum(total_sales#16)#47] Results [2]: [i_item_id#10, sum(total_sales#16)#47 AS total_sales#48] -(71) TakeOrderedAndProject +(63) TakeOrderedAndProject Input [2]: [i_item_id#10, total_sales#48] Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#10, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (68) ++- * ColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) (unknown) Scan parquet spark_catalog.default.date_dim @@ -415,23 +383,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(65) CometFilter Input [3]: [d_date_sk#6, d_year#49, d_moy#50] Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2001)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#6)) -(74) CometProject +(66) CometProject Input [3]: [d_date_sk#6, d_year#49, d_moy#50] Arguments: [d_date_sk#6], [d_date_sk#6] -(75) ColumnarToRow [codegen id : 1] +(67) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(76) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 6f7092ce81..f781ed1f7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -1,113 +1,105 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (20) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - RowToColumnar - WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (19) + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #2 + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_item_id] - CometFilter [i_color] - CometScan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [ca_address_sk] #4 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #8 - RowToColumnar - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 + CometProject [i_item_id] + CometFilter [i_color] + CometScan parquet spark_catalog.default.item [i_item_id,i_color] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (11) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + WholeStageCodegen (18) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (17) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index 87a2c73e68..8302389585 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -1,57 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (53) -+- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (32) - : : +- * Filter (31) - : : +- Window (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * ColumnarToRow (27) - : : +- CometSort (26) - : : +- CometColumnarExchange (25) - : : +- RowToColumnar (24) - : : +- * HashAggregate (23) - : : +- * ColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- RowToColumnar (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (42) - : +- * Project (41) - : +- Window (40) - : +- * ColumnarToRow (39) - : +- CometSort (38) - : +- CometColumnarExchange (37) - : +- RowToColumnar (36) - : +- * HashAggregate (35) - : +- * ColumnarToRow (34) - : +- ReusedExchange (33) - +- BroadcastExchange (50) - +- * Project (49) - +- Window (48) - +- * ColumnarToRow (47) - +- CometSort (46) - +- ReusedExchange (45) +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * Filter (25) + : : +- Window (24) + : : +- * Sort (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- Exchange (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.call_center (13) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) (unknown) Scan parquet spark_catalog.default.item @@ -97,7 +89,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 57] +(10) ReusedExchange [Reuses operator id: 49] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -145,149 +137,125 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#6))] Aggregate Attributes [1]: [sum#14] Results [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -(20) RowToColumnar +(20) Exchange Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometColumnarExchange -Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) ColumnarToRow [codegen id : 5] -Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] - -(23) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 5] Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#16] Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS _w0#18] -(24) RowToColumnar -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] - -(25) CometColumnarExchange +(22) Exchange Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(26) CometSort +(23) Sort [codegen id : 6] Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 -(27) ColumnarToRow [codegen id : 6] -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] - -(28) Window +(24) Window Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(29) Filter [codegen id : 7] +(25) Filter [codegen id : 7] Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(30) Window +(26) Window Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] -(31) Filter [codegen id : 22] +(27) Filter [codegen id : 22] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(32) Project [codegen id : 22] +(28) Project [codegen id : 22] Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -(33) ReusedExchange [Reuses operator id: 21] +(29) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(34) ColumnarToRow [codegen id : 12] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] - -(35) HashAggregate [codegen id : 12] +(30) HashAggregate [codegen id : 12] Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] -(36) RowToColumnar -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] - -(37) CometColumnarExchange +(31) Exchange Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) CometSort +(32) Sort [codegen id : 13] Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 -(39) ColumnarToRow [codegen id : 13] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] - -(40) Window +(33) Window Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(41) Project [codegen id : 14] +(34) Project [codegen id : 14] Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] -(42) BroadcastExchange +(35) BroadcastExchange Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 22] +(36) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] +(37) Project [codegen id : 22] Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -(45) ReusedExchange [Reuses operator id: 37] +(38) ReusedExchange [Reuses operator id: 31] Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -(46) CometSort +(39) Sort [codegen id : 20] Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 -(47) ColumnarToRow [codegen id : 20] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] - -(48) Window +(40) Window Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(49) Project [codegen id : 21] +(41) Project [codegen id : 21] Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] -(50) BroadcastExchange +(42) BroadcastExchange Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 22] +(43) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(52) Project [codegen id : 22] +(44) Project [codegen id : 22] Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -(53) TakeOrderedAndProject +(45) TakeOrderedAndProject Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (57) -+- * ColumnarToRow (56) - +- CometFilter (55) - +- CometScan parquet spark_catalog.default.date_dim (54) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim @@ -297,14 +265,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(55) CometFilter +(47) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(56) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(57) BroadcastExchange +(49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index a68dc18ad2..3bc01343ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -13,52 +13,48 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (6) - ColumnarToRow + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + Exchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometFilter [cs_item_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 WholeStageCodegen (14) @@ -66,16 +62,13 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (13) - ColumnarToRow + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - RowToColumnar - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + Exchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 WholeStageCodegen (21) @@ -83,7 +76,6 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (20) - ColumnarToRow + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index 49924f25d3..e3b68cabe2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -1,59 +1,53 @@ == Physical Plan == -TakeOrderedAndProject (55) -+- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (18) - : : +- * HashAggregate (17) - : : +- * ColumnarToRow (16) - : : +- CometColumnarExchange (15) - : : +- RowToColumnar (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (10) - : +- BroadcastExchange (34) - : +- * Filter (33) - : +- * HashAggregate (32) - : +- * ColumnarToRow (31) - : +- CometColumnarExchange (30) - : +- RowToColumnar (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * ColumnarToRow (21) - : : : +- CometFilter (20) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (22) - : +- ReusedExchange (25) - +- BroadcastExchange (52) - +- * Filter (51) - +- * HashAggregate (50) - +- * ColumnarToRow (49) - +- CometColumnarExchange (48) - +- RowToColumnar (47) - +- * HashAggregate (46) - +- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * ColumnarToRow (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (40) - +- ReusedExchange (43) +TakeOrderedAndProject (49) ++- * Project (48) + +- * BroadcastHashJoin Inner BuildRight (47) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Filter (16) + : : +- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- ReusedExchange (10) + : +- BroadcastExchange (30) + : +- * Filter (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometFilter (18) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (17) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (46) + +- * Filter (45) + +- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * ColumnarToRow (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.web_sales (33) + : +- ReusedExchange (36) + +- ReusedExchange (39) (unknown) Scan parquet spark_catalog.default.store_sales @@ -99,7 +93,7 @@ Join condition: None Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6] -(10) ReusedExchange [Reuses operator id: 66] +(10) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#7] (11) BroadcastHashJoin [codegen id : 4] @@ -119,24 +113,18 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#8] Results [2]: [i_item_id#6, sum#9] -(14) RowToColumnar +(14) Exchange Input [2]: [i_item_id#6, sum#9] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [2]: [i_item_id#6, sum#9] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 15] -Input [2]: [i_item_id#6, sum#9] - -(17) HashAggregate [codegen id : 15] +(15) HashAggregate [codegen id : 15] Input [2]: [i_item_id#6, sum#9] Keys [1]: [i_item_id#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#10] Results [2]: [i_item_id#6 AS item_id#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#10,17,2) AS ss_item_rev#12] -(18) Filter [codegen id : 15] +(16) Filter [codegen id : 15] Input [2]: [item_id#11, ss_item_rev#12] Condition : isnotnull(ss_item_rev#12) @@ -148,78 +136,72 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(20) CometFilter +(18) CometFilter Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] Condition : isnotnull(cs_item_sk#13) -(21) ColumnarToRow [codegen id : 8] +(19) ColumnarToRow [codegen id : 8] Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] -(22) ReusedExchange [Reuses operator id: 7] +(20) ReusedExchange [Reuses operator id: 7] Output [2]: [i_item_sk#17, i_item_id#18] -(23) BroadcastHashJoin [codegen id : 8] +(21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#13] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(22) Project [codegen id : 8] Output [3]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18] Input [5]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_sk#17, i_item_id#18] -(25) ReusedExchange [Reuses operator id: 66] +(23) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#19] -(26) BroadcastHashJoin [codegen id : 8] +(24) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#15] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(27) Project [codegen id : 8] +(25) Project [codegen id : 8] Output [2]: [cs_ext_sales_price#14, i_item_id#18] Input [4]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18, d_date_sk#19] -(28) HashAggregate [codegen id : 8] +(26) HashAggregate [codegen id : 8] Input [2]: [cs_ext_sales_price#14, i_item_id#18] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#14))] Aggregate Attributes [1]: [sum#20] Results [2]: [i_item_id#18, sum#21] -(29) RowToColumnar -Input [2]: [i_item_id#18, sum#21] - -(30) CometColumnarExchange +(27) Exchange Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(31) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_id#18, sum#21] - -(32) HashAggregate [codegen id : 9] +(28) HashAggregate [codegen id : 9] Input [2]: [i_item_id#18, sum#21] Keys [1]: [i_item_id#18] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#14))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#14))#22] Results [2]: [i_item_id#18 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#14))#22,17,2) AS cs_item_rev#24] -(33) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [2]: [item_id#23, cs_item_rev#24] Condition : isnotnull(cs_item_rev#24) -(34) BroadcastExchange +(30) BroadcastExchange Input [2]: [item_id#23, cs_item_rev#24] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(35) BroadcastHashJoin [codegen id : 15] +(31) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#11] Right keys [1]: [item_id#23] Join type: Inner Join condition: ((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) -(36) Project [codegen id : 15] +(32) Project [codegen id : 15] Output [3]: [item_id#11, ss_item_rev#12, cs_item_rev#24] Input [4]: [item_id#11, ss_item_rev#12, item_id#23, cs_item_rev#24] @@ -231,99 +213,93 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(38) CometFilter +(34) CometFilter Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] Condition : isnotnull(ws_item_sk#25) -(39) ColumnarToRow [codegen id : 13] +(35) ColumnarToRow [codegen id : 13] Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -(40) ReusedExchange [Reuses operator id: 7] +(36) ReusedExchange [Reuses operator id: 7] Output [2]: [i_item_sk#29, i_item_id#30] -(41) BroadcastHashJoin [codegen id : 13] +(37) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_item_sk#25] Right keys [1]: [i_item_sk#29] Join type: Inner Join condition: None -(42) Project [codegen id : 13] +(38) Project [codegen id : 13] Output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#29, i_item_id#30] -(43) ReusedExchange [Reuses operator id: 66] +(39) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#31] -(44) BroadcastHashJoin [codegen id : 13] +(40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#27] Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(45) Project [codegen id : 13] +(41) Project [codegen id : 13] Output [2]: [ws_ext_sales_price#26, i_item_id#30] Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30, d_date_sk#31] -(46) HashAggregate [codegen id : 13] +(42) HashAggregate [codegen id : 13] Input [2]: [ws_ext_sales_price#26, i_item_id#30] Keys [1]: [i_item_id#30] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] Aggregate Attributes [1]: [sum#32] Results [2]: [i_item_id#30, sum#33] -(47) RowToColumnar -Input [2]: [i_item_id#30, sum#33] - -(48) CometColumnarExchange -Input [2]: [i_item_id#30, sum#33] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(49) ColumnarToRow [codegen id : 14] +(43) Exchange Input [2]: [i_item_id#30, sum#33] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(50) HashAggregate [codegen id : 14] +(44) HashAggregate [codegen id : 14] Input [2]: [i_item_id#30, sum#33] Keys [1]: [i_item_id#30] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#34] Results [2]: [i_item_id#30 AS item_id#35, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#34,17,2) AS ws_item_rev#36] -(51) Filter [codegen id : 14] +(45) Filter [codegen id : 14] Input [2]: [item_id#35, ws_item_rev#36] Condition : isnotnull(ws_item_rev#36) -(52) BroadcastExchange +(46) BroadcastExchange Input [2]: [item_id#35, ws_item_rev#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(53) BroadcastHashJoin [codegen id : 15] +(47) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#11] Right keys [1]: [item_id#35] Join type: Inner Join condition: ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * ws_item_rev#36)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) -(54) Project [codegen id : 15] +(48) Project [codegen id : 15] Output [8]: [item_id#11, ss_item_rev#12, (((ss_item_rev#12 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ss_dev#37, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS cs_dev#38, ws_item_rev#36, (((ws_item_rev#36 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ws_dev#39, (((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36) / 3) AS average#40] Input [5]: [item_id#11, ss_item_rev#12, cs_item_rev#24, item_id#35, ws_item_rev#36] -(55) TakeOrderedAndProject +(49) TakeOrderedAndProject Input [8]: [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] Arguments: 100, [item_id#11 ASC NULLS FIRST, ss_item_rev#12 ASC NULLS FIRST], [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (66) -+- * Project (65) - +- * BroadcastHashJoin LeftSemi BuildRight (64) - :- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.date_dim (56) - +- BroadcastExchange (63) - +- * ColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.date_dim (59) +BroadcastExchange (60) ++- * Project (59) + +- * BroadcastHashJoin LeftSemi BuildRight (58) + :- * ColumnarToRow (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.date_dim (50) + +- BroadcastExchange (57) + +- * ColumnarToRow (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) (unknown) Scan parquet spark_catalog.default.date_dim @@ -333,11 +309,11 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter +(51) CometFilter Input [2]: [d_date_sk#7, d_date#41] Condition : isnotnull(d_date_sk#7) -(58) ColumnarToRow [codegen id : 2] +(52) ColumnarToRow [codegen id : 2] Input [2]: [d_date_sk#7, d_date#41] (unknown) Scan parquet spark_catalog.default.date_dim @@ -347,40 +323,40 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(60) CometFilter +(54) CometFilter Input [2]: [d_date#42, d_week_seq#43] Condition : (isnotnull(d_week_seq#43) AND (d_week_seq#43 = Subquery scalar-subquery#44, [id=#45])) -(61) CometProject +(55) CometProject Input [2]: [d_date#42, d_week_seq#43] Arguments: [d_date#42], [d_date#42] -(62) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [1]: [d_date#42] -(63) BroadcastExchange +(57) BroadcastExchange Input [1]: [d_date#42] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=7] -(64) BroadcastHashJoin [codegen id : 2] +(58) BroadcastHashJoin [codegen id : 2] Left keys [1]: [d_date#41] Right keys [1]: [d_date#42] Join type: LeftSemi Join condition: None -(65) Project [codegen id : 2] +(59) Project [codegen id : 2] Output [1]: [d_date_sk#7] Input [2]: [d_date_sk#7, d_date#41] -(66) BroadcastExchange +(60) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 60 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* ColumnarToRow (70) -+- CometProject (69) - +- CometFilter (68) - +- CometScan parquet spark_catalog.default.date_dim (67) +Subquery:2 Hosting operator id = 54 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* ColumnarToRow (64) ++- CometProject (63) + +- CometFilter (62) + +- CometScan parquet spark_catalog.default.date_dim (61) (unknown) Scan parquet spark_catalog.default.date_dim @@ -390,19 +366,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(68) CometFilter +(62) CometFilter Input [2]: [d_date#46, d_week_seq#47] Condition : (isnotnull(d_date#46) AND (d_date#46 = 2000-01-03)) -(69) CometProject +(63) CometProject Input [2]: [d_date#46, d_week_seq#47] Arguments: [d_week_seq#47], [d_week_seq#47] -(70) ColumnarToRow [codegen id : 1] +(64) ColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#47] -Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index efc0244704..d3bb0dc388 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -6,98 +6,92 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] Filter [ss_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (2) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - ColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date] - CometFilter [d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (2) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + ColumnarToRow + InputAdapter + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date] + CometFilter [d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_date] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 WholeStageCodegen (9) Filter [cs_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #6 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (14) - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #8 - RowToColumnar - WholeStageCodegen (13) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (8) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (14) + Filter [ws_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] + InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (13) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 + InputAdapter + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt index 1211840ec0..62b3f58686 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt @@ -1,51 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * HashAggregate (14) - : : : +- * ColumnarToRow (13) - : : : +- CometColumnarExchange (12) - : : : +- RowToColumnar (11) - : : : +- * HashAggregate (10) - : : : +- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.date_dim (4) - : : +- BroadcastExchange (18) - : : +- * ColumnarToRow (17) - : : +- CometFilter (16) - : : +- CometScan parquet spark_catalog.default.store (15) - : +- BroadcastExchange (25) - : +- * ColumnarToRow (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometScan parquet spark_catalog.default.date_dim (21) - +- BroadcastExchange (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * HashAggregate (30) - : : +- * ColumnarToRow (29) - : : +- ReusedExchange (28) - : +- BroadcastExchange (34) - : +- * ColumnarToRow (33) - : +- CometFilter (32) - : +- CometScan parquet spark_catalog.default.store (31) - +- BroadcastExchange (41) - +- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) +TakeOrderedAndProject (44) ++- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * HashAggregate (12) + : : : +- Exchange (11) + : : : +- * HashAggregate (10) + : : : +- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometProject (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.date_dim (19) + +- BroadcastExchange (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * HashAggregate (27) + : : +- ReusedExchange (26) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.store (28) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) (unknown) Scan parquet spark_catalog.default.store_sales @@ -98,17 +95,11 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) T Aggregate Attributes [7]: [sum#7, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13] Results [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] -(11) RowToColumnar +(11) Exchange Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] +Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(12) CometColumnarExchange -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(13) ColumnarToRow [codegen id : 10] -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] - -(14) HashAggregate [codegen id : 10] +(12) HashAggregate [codegen id : 10] Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] @@ -122,24 +113,24 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] Condition : (isnotnull(s_store_sk#35) AND isnotnull(s_store_id#36)) -(17) ColumnarToRow [codegen id : 3] +(15) ColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] -(18) BroadcastExchange +(16) BroadcastExchange Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 10] +(17) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#35] Join type: Inner Join condition: None -(20) Project [codegen id : 10] +(18) Project [codegen id : 10] Output [10]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37] Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_sk#35, s_store_id#36, s_store_name#37] @@ -150,38 +141,35 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] ReadSchema: struct -(22) CometFilter +(20) CometFilter Input [2]: [d_month_seq#38, d_week_seq#39] Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= 1212)) AND (d_month_seq#38 <= 1223)) AND isnotnull(d_week_seq#39)) -(23) CometProject +(21) CometProject Input [2]: [d_month_seq#38, d_week_seq#39] Arguments: [d_week_seq#39], [d_week_seq#39] -(24) ColumnarToRow [codegen id : 4] +(22) ColumnarToRow [codegen id : 4] Input [1]: [d_week_seq#39] -(25) BroadcastExchange +(23) BroadcastExchange Input [1]: [d_week_seq#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(26) BroadcastHashJoin [codegen id : 10] +(24) BroadcastHashJoin [codegen id : 10] Left keys [1]: [d_week_seq#5] Right keys [1]: [d_week_seq#39] Join type: Inner Join condition: None -(27) Project [codegen id : 10] +(25) Project [codegen id : 10] Output [10]: [s_store_name#37 AS s_store_name1#40, d_week_seq#5 AS d_week_seq1#41, s_store_id#36 AS s_store_id1#42, sun_sales#28 AS sun_sales1#43, mon_sales#29 AS mon_sales1#44, tue_sales#30 AS tue_sales1#45, wed_sales#31 AS wed_sales1#46, thu_sales#32 AS thu_sales1#47, fri_sales#33 AS fri_sales1#48, sat_sales#34 AS sat_sales1#49] Input [11]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37, d_week_seq#39] -(28) ReusedExchange [Reuses operator id: 12] +(26) ReusedExchange [Reuses operator id: 11] Output [9]: [d_week_seq#5, ss_store_sk#1, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] -(29) ColumnarToRow [codegen id : 9] -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] - -(30) HashAggregate [codegen id : 9] +(27) HashAggregate [codegen id : 9] Input [9]: [d_week_seq#5, ss_store_sk#1, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] @@ -195,24 +183,24 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(32) CometFilter +(29) CometFilter Input [2]: [s_store_sk#57, s_store_id#58] Condition : (isnotnull(s_store_sk#57) AND isnotnull(s_store_id#58)) -(33) ColumnarToRow [codegen id : 7] +(30) ColumnarToRow [codegen id : 7] Input [2]: [s_store_sk#57, s_store_id#58] -(34) BroadcastExchange +(31) BroadcastExchange Input [2]: [s_store_sk#57, s_store_id#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(35) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#57] Join type: Inner Join condition: None -(36) Project [codegen id : 9] +(33) Project [codegen id : 9] Output [9]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#58] Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_sk#57, s_store_id#58] @@ -223,46 +211,46 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] ReadSchema: struct -(38) CometFilter +(35) CometFilter Input [2]: [d_month_seq#59, d_week_seq#60] Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1224)) AND (d_month_seq#59 <= 1235)) AND isnotnull(d_week_seq#60)) -(39) CometProject +(36) CometProject Input [2]: [d_month_seq#59, d_week_seq#60] Arguments: [d_week_seq#60], [d_week_seq#60] -(40) ColumnarToRow [codegen id : 8] +(37) ColumnarToRow [codegen id : 8] Input [1]: [d_week_seq#60] -(41) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_week_seq#60] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(42) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [d_week_seq#5] Right keys [1]: [d_week_seq#60] Join type: Inner Join condition: None -(43) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [9]: [d_week_seq#5 AS d_week_seq2#61, s_store_id#58 AS s_store_id2#62, sun_sales#28 AS sun_sales2#63, mon_sales#29 AS mon_sales2#64, tue_sales#30 AS tue_sales2#65, wed_sales#31 AS wed_sales2#66, thu_sales#32 AS thu_sales2#67, fri_sales#33 AS fri_sales2#68, sat_sales#34 AS sat_sales2#69] Input [10]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#58, d_week_seq#60] -(44) BroadcastExchange +(41) BroadcastExchange Input [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=7] -(45) BroadcastHashJoin [codegen id : 10] +(42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#42, d_week_seq1#41] Right keys [2]: [s_store_id2#62, (d_week_seq2#61 - 52)] Join type: Inner Join condition: None -(46) Project [codegen id : 10] +(43) Project [codegen id : 10] Output [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1#43 / sun_sales2#63) AS (sun_sales1 / sun_sales2)#70, (mon_sales1#44 / mon_sales2#64) AS (mon_sales1 / mon_sales2)#71, (tue_sales1#45 / tue_sales2#65) AS (tue_sales1 / tue_sales2)#72, (wed_sales1#46 / wed_sales2#66) AS (wed_sales1 / wed_sales2)#73, (thu_sales1#47 / thu_sales2#67) AS (thu_sales1 / thu_sales2)#74, (fri_sales1#48 / fri_sales2#68) AS (fri_sales1 / fri_sales2)#75, (sat_sales1#49 / sat_sales2#69) AS (sat_sales1 / sat_sales2)#76] Input [19]: [s_store_name1#40, d_week_seq1#41, s_store_id1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -(47) TakeOrderedAndProject +(44) TakeOrderedAndProject Input [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] Arguments: 100, [s_store_name1#40 ASC NULLS FIRST, s_store_id1#42 ASC NULLS FIRST, d_week_seq1#41 ASC NULLS FIRST], [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index 84c312a617..9ad61e946e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -7,25 +7,23 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] BroadcastHashJoin [ss_store_sk,s_store_sk] HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [d_week_seq,ss_store_sk] #1 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + InputAdapter + Exchange [d_week_seq,ss_store_sk] #1 + WholeStageCodegen (2) + HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) @@ -49,9 +47,8 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 + InputAdapter + ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 InputAdapter BroadcastExchange #6 WholeStageCodegen (7) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index 1e70914b3b..cc4720a192 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -1,45 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Filter (40) - +- * HashAggregate (39) - +- * ColumnarToRow (38) - +- CometColumnarExchange (37) - +- RowToColumnar (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer_address (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometFilter (27) - +- CometHashAggregate (26) - +- CometColumnarExchange (25) - +- CometHashAggregate (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.item (22) +TakeOrderedAndProject (39) ++- * Filter (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer_address (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store_sales (10) + : +- ReusedExchange (16) + +- BroadcastExchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * ColumnarToRow (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- BroadcastExchange (29) + +- * Filter (28) + +- * HashAggregate (27) + +- Exchange (26) + +- * ColumnarToRow (25) + +- CometHashAggregate (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.item (22) (unknown) Scan parquet spark_catalog.default.customer_address @@ -53,7 +51,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#1, ca_state#2] Condition : isnotnull(ca_address_sk#1) -(3) ColumnarToRow [codegen id : 6] +(3) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#1, ca_state#2] (unknown) Scan parquet spark_catalog.default.customer @@ -74,13 +72,13 @@ Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 6] +(8) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ca_address_sk#1] Right keys [1]: [c_current_addr_sk#4] Join type: Inner Join condition: None -(9) Project [codegen id : 6] +(9) Project [codegen id : 7] Output [2]: [ca_state#2, c_customer_sk#3] Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4] @@ -103,26 +101,26 @@ Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 6] +(14) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: Inner Join condition: None -(15) Project [codegen id : 6] +(15) Project [codegen id : 7] Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -(16) ReusedExchange [Reuses operator id: 46] +(16) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#9] -(17) BroadcastHashJoin [codegen id : 6] +(17) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(18) Project [codegen id : 6] +(18) Project [codegen id : 7] Output [2]: [ca_state#2, ss_item_sk#5] Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] @@ -137,7 +135,7 @@ ReadSchema: struct Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10)) -(21) ColumnarToRow [codegen id : 5] +(21) ColumnarToRow [codegen id : 6] Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] (unknown) Scan parquet spark_catalog.default.item @@ -156,154 +154,152 @@ Input [2]: [i_current_price#13, i_category#14] Keys [1]: [i_category#14] Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] -(25) CometColumnarExchange +(25) ColumnarToRow [codegen id : 4] Input [3]: [i_category#14, sum#15, count#16] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(26) CometHashAggregate +(26) Exchange +Input [3]: [i_category#14, sum#15, count#16] +Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 5] Input [3]: [i_category#14, sum#15, count#16] Keys [1]: [i_category#14] Functions [1]: [avg(UnscaledValue(i_current_price#13))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#17] +Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#17 / 100.0) as decimal(11,6)) AS avg(i_current_price)#18, i_category#14] -(27) CometFilter -Input [2]: [avg(i_current_price)#17, i_category#14] -Condition : isnotnull(avg(i_current_price)#17) - -(28) ColumnarToRow [codegen id : 4] -Input [2]: [avg(i_current_price)#17, i_category#14] +(28) Filter [codegen id : 5] +Input [2]: [avg(i_current_price)#18, i_category#14] +Condition : isnotnull(avg(i_current_price)#18) (29) BroadcastExchange -Input [2]: [avg(i_current_price)#17, i_category#14] +Input [2]: [avg(i_current_price)#18, i_category#14] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] -(30) BroadcastHashJoin [codegen id : 5] +(30) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#12] Right keys [1]: [i_category#14] Join type: Inner -Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#17)) +Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#18)) -(31) Project [codegen id : 5] +(31) Project [codegen id : 6] Output [1]: [i_item_sk#10] -Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#17, i_category#14] +Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#18, i_category#14] (32) BroadcastExchange Input [1]: [i_item_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 6] +(33) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#5] Right keys [1]: [i_item_sk#10] Join type: Inner Join condition: None -(34) Project [codegen id : 6] +(34) Project [codegen id : 7] Output [1]: [ca_state#2] Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#10] -(35) HashAggregate [codegen id : 6] +(35) HashAggregate [codegen id : 7] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_state#2, count#19] - -(36) RowToColumnar -Input [2]: [ca_state#2, count#19] +Aggregate Attributes [1]: [count#19] +Results [2]: [ca_state#2, count#20] -(37) CometColumnarExchange -Input [2]: [ca_state#2, count#19] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(36) Exchange +Input [2]: [ca_state#2, count#20] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(38) ColumnarToRow [codegen id : 7] -Input [2]: [ca_state#2, count#19] - -(39) HashAggregate [codegen id : 7] -Input [2]: [ca_state#2, count#19] +(37) HashAggregate [codegen id : 8] +Input [2]: [ca_state#2, count#20] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#20] -Results [2]: [ca_state#2 AS state#21, count(1)#20 AS cnt#22] +Aggregate Attributes [1]: [count(1)#21] +Results [2]: [ca_state#2 AS state#22, count(1)#21 AS cnt#23] -(40) Filter [codegen id : 7] -Input [2]: [state#21, cnt#22] -Condition : (cnt#22 >= 10) +(38) Filter [codegen id : 8] +Input [2]: [state#22, cnt#23] +Condition : (cnt#23 >= 10) -(41) TakeOrderedAndProject -Input [2]: [state#21, cnt#22] -Arguments: 100, [cnt#22 ASC NULLS FIRST], [state#21, cnt#22] +(39) TakeOrderedAndProject +Input [2]: [state#22, cnt#23] +Arguments: 100, [cnt#23 ASC NULLS FIRST], [state#22, cnt#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (46) -+- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.date_dim (42) +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#23] +Output [2]: [d_date_sk#9, d_month_seq#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#23] -Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-subquery#24, [id=#25])) AND isnotnull(d_date_sk#9)) +(41) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#24] +Condition : ((isnotnull(d_month_seq#24) AND (d_month_seq#24 = Subquery scalar-subquery#25, [id=#26])) AND isnotnull(d_date_sk#9)) -(44) CometProject -Input [2]: [d_date_sk#9, d_month_seq#23] +(42) CometProject +Input [2]: [d_date_sk#9, d_month_seq#24] Arguments: [d_date_sk#9], [d_date_sk#9] -(45) ColumnarToRow [codegen id : 1] +(43) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(46) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#24, [id=#25] -* ColumnarToRow (53) -+- CometHashAggregate (52) - +- CometColumnarExchange (51) - +- CometHashAggregate (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan parquet spark_catalog.default.date_dim (47) +Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#25, [id=#26] +* HashAggregate (51) ++- Exchange (50) + +- * ColumnarToRow (49) + +- CometHashAggregate (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] +Output [3]: [d_month_seq#27, d_year#28, d_moy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(48) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) +(46) CometFilter +Input [3]: [d_month_seq#27, d_year#28, d_moy#29] +Condition : (((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2000)) AND (d_moy#29 = 1)) -(49) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] +(47) CometProject +Input [3]: [d_month_seq#27, d_year#28, d_moy#29] +Arguments: [d_month_seq#27], [d_month_seq#27] -(50) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +(48) CometHashAggregate +Input [1]: [d_month_seq#27] +Keys [1]: [d_month_seq#27] Functions: [] -(51) CometColumnarExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +(49) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#27] -(52) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] +(50) Exchange +Input [1]: [d_month_seq#27] +Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(53) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] +(51) HashAggregate [codegen id : 2] +Input [1]: [d_month_seq#27] +Keys [1]: [d_month_seq#27] +Functions: [] +Aggregate Attributes: [] +Results [1]: [d_month_seq#27] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index fe28888f0f..fc6d5d0127 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -1,76 +1,78 @@ TakeOrderedAndProject [cnt,state] - WholeStageCodegen (7) + WholeStageCodegen (8) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_state] #1 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + Exchange [ca_state] #1 + WholeStageCodegen (7) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ca_state,ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ca_state,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [ca_state,c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometColumnarExchange [d_month_seq] #5 + CometFilter [c_current_addr_sk,c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (2) + HashAggregate [d_month_seq] + InputAdapter + Exchange [d_month_seq] #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] CometFilter [d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow - InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [avg(i_current_price)] - CometHashAggregate [i_category,sum,count] - CometColumnarExchange [i_category] #8 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Project [i_item_sk] + BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] + ColumnarToRow + InputAdapter + CometFilter [i_current_price,i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + Filter [avg(i_current_price)] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + InputAdapter + Exchange [i_category] #8 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter CometHashAggregate [i_category,i_current_price] CometFilter [i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 63dd79e72a..8885bc8e5b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -1,75 +1,67 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * HashAggregate (70) - +- * ColumnarToRow (69) - +- CometColumnarExchange (68) - +- RowToColumnar (67) - +- * HashAggregate (66) - +- Union (65) - :- * HashAggregate (30) - : +- * ColumnarToRow (29) - : +- CometColumnarExchange (28) - : +- RowToColumnar (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - :- * HashAggregate (47) - : +- * ColumnarToRow (46) - : +- CometColumnarExchange (45) - : +- RowToColumnar (44) - : +- * HashAggregate (43) - : +- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (36) - : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : :- * ColumnarToRow (33) - : : : : +- CometFilter (32) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : : +- ReusedExchange (34) - : : +- ReusedExchange (37) - : +- ReusedExchange (40) - +- * HashAggregate (64) - +- * ColumnarToRow (63) - +- CometColumnarExchange (62) - +- RowToColumnar (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * ColumnarToRow (50) - : : : +- CometFilter (49) - : : : +- CometScan parquet spark_catalog.default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) +TakeOrderedAndProject (63) ++- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * ColumnarToRow (31) + : : : : +- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * ColumnarToRow (46) + : : : +- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) (unknown) Scan parquet spark_catalog.default.store_sales @@ -87,7 +79,7 @@ Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 5] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 76] +(4) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 5] @@ -195,17 +187,11 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#13] Results [2]: [i_item_id#10, sum#14] -(27) RowToColumnar +(27) Exchange Input [2]: [i_item_id#10, sum#14] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) CometColumnarExchange -Input [2]: [i_item_id#10, sum#14] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_id#10, sum#14] - -(30) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 6] Input [2]: [i_item_id#10, sum#14] Keys [1]: [i_item_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -220,70 +206,64 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(32) CometFilter +(30) CometFilter Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) -(33) ColumnarToRow [codegen id : 11] +(31) ColumnarToRow [codegen id : 11] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -(34) ReusedExchange [Reuses operator id: 76] +(32) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#22] -(35) BroadcastHashJoin [codegen id : 11] +(33) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#20] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(36) Project [codegen id : 11] +(34) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -(37) ReusedExchange [Reuses operator id: 11] +(35) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#23] -(38) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#17] Right keys [1]: [ca_address_sk#23] Join type: Inner Join condition: None -(39) Project [codegen id : 11] +(37) Project [codegen id : 11] Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] -(40) ReusedExchange [Reuses operator id: 23] +(38) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#24, i_item_id#25] -(41) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#18] Right keys [1]: [i_item_sk#24] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(40) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#19, i_item_id#25] Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] -(43) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#19, i_item_id#25] Keys [1]: [i_item_id#25] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] Aggregate Attributes [1]: [sum#26] Results [2]: [i_item_id#25, sum#27] -(44) RowToColumnar -Input [2]: [i_item_id#25, sum#27] - -(45) CometColumnarExchange -Input [2]: [i_item_id#25, sum#27] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(46) ColumnarToRow [codegen id : 12] +(42) Exchange Input [2]: [i_item_id#25, sum#27] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(47) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 12] Input [2]: [i_item_id#25, sum#27] Keys [1]: [i_item_id#25] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] @@ -298,114 +278,102 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) CometFilter +(45) CometFilter Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) -(50) ColumnarToRow [codegen id : 17] +(46) ColumnarToRow [codegen id : 17] Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -(51) ReusedExchange [Reuses operator id: 76] +(47) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#35] -(52) BroadcastHashJoin [codegen id : 17] +(48) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_sold_date_sk#33] Right keys [1]: [d_date_sk#35] Join type: Inner Join condition: None -(53) Project [codegen id : 17] +(49) Project [codegen id : 17] Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] -(54) ReusedExchange [Reuses operator id: 11] +(50) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#36] -(55) BroadcastHashJoin [codegen id : 17] +(51) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_bill_addr_sk#31] Right keys [1]: [ca_address_sk#36] Join type: Inner Join condition: None -(56) Project [codegen id : 17] +(52) Project [codegen id : 17] Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -(57) ReusedExchange [Reuses operator id: 23] +(53) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#37, i_item_id#38] -(58) BroadcastHashJoin [codegen id : 17] +(54) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_item_sk#30] Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(55) Project [codegen id : 17] Output [2]: [ws_ext_sales_price#32, i_item_id#38] Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] -(60) HashAggregate [codegen id : 17] +(56) HashAggregate [codegen id : 17] Input [2]: [ws_ext_sales_price#32, i_item_id#38] Keys [1]: [i_item_id#38] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] Aggregate Attributes [1]: [sum#39] Results [2]: [i_item_id#38, sum#40] -(61) RowToColumnar +(57) Exchange Input [2]: [i_item_id#38, sum#40] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(62) CometColumnarExchange -Input [2]: [i_item_id#38, sum#40] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(63) ColumnarToRow [codegen id : 18] -Input [2]: [i_item_id#38, sum#40] - -(64) HashAggregate [codegen id : 18] +(58) HashAggregate [codegen id : 18] Input [2]: [i_item_id#38, sum#40] Keys [1]: [i_item_id#38] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] -(65) Union +(59) Union -(66) HashAggregate [codegen id : 19] +(60) HashAggregate [codegen id : 19] Input [2]: [i_item_id#10, total_sales#16] Keys [1]: [i_item_id#10] Functions [1]: [partial_sum(total_sales#16)] Aggregate Attributes [2]: [sum#43, isEmpty#44] Results [3]: [i_item_id#10, sum#45, isEmpty#46] -(67) RowToColumnar -Input [3]: [i_item_id#10, sum#45, isEmpty#46] - -(68) CometColumnarExchange -Input [3]: [i_item_id#10, sum#45, isEmpty#46] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(69) ColumnarToRow [codegen id : 20] +(61) Exchange Input [3]: [i_item_id#10, sum#45, isEmpty#46] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(70) HashAggregate [codegen id : 20] +(62) HashAggregate [codegen id : 20] Input [3]: [i_item_id#10, sum#45, isEmpty#46] Keys [1]: [i_item_id#10] Functions [1]: [sum(total_sales#16)] Aggregate Attributes [1]: [sum(total_sales#16)#47] Results [2]: [i_item_id#10, sum(total_sales#16)#47 AS total_sales#48] -(71) TakeOrderedAndProject +(63) TakeOrderedAndProject Input [2]: [i_item_id#10, total_sales#48] Arguments: 100, [i_item_id#10 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#10, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (68) ++- * ColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) (unknown) Scan parquet spark_catalog.default.date_dim @@ -415,23 +383,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(65) CometFilter Input [3]: [d_date_sk#6, d_year#49, d_moy#50] Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 9)) AND isnotnull(d_date_sk#6)) -(74) CometProject +(66) CometProject Input [3]: [d_date_sk#6, d_year#49, d_moy#50] Arguments: [d_date_sk#6], [d_date_sk#6] -(75) ColumnarToRow [codegen id : 1] +(67) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(76) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index f2585fe939..b010414a86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -1,113 +1,105 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (20) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - RowToColumnar - WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_addr_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (19) + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #2 + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [i_item_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [ca_address_sk] #4 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #8 - RowToColumnar - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_addr_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 + CometProject [i_item_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_item_id,i_category] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (11) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + WholeStageCodegen (18) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (17) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index 391b00f0ad..ee9a8409a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -1,75 +1,71 @@ == Physical Plan == -* Project (71) -+- * BroadcastNestedLoopJoin Inner BuildRight (70) - :- * HashAggregate (45) - : +- * ColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- RowToColumnar (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (26) - : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : :- * Project (20) - : : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * ColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan parquet spark_catalog.default.promotion (11) - : : : : +- ReusedExchange (18) - : : : +- BroadcastExchange (24) - : : : +- * ColumnarToRow (23) - : : : +- CometFilter (22) - : : : +- CometScan parquet spark_catalog.default.customer (21) - : : +- BroadcastExchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.customer_address (27) - : +- BroadcastExchange (38) - : +- * ColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan parquet spark_catalog.default.item (34) - +- BroadcastExchange (69) - +- * HashAggregate (68) - +- * ColumnarToRow (67) - +- CometColumnarExchange (66) - +- RowToColumnar (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * Project (60) - : +- * BroadcastHashJoin Inner BuildRight (59) - : :- * Project (57) - : : +- * BroadcastHashJoin Inner BuildRight (56) - : : :- * Project (54) - : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : :- * Project (51) - : : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : : :- * ColumnarToRow (48) - : : : : : +- CometFilter (47) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (46) - : : : : +- ReusedExchange (49) - : : : +- ReusedExchange (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (58) - +- ReusedExchange (61) +* Project (67) ++- * BroadcastNestedLoopJoin Inner BuildRight (66) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (33) + : : +- * BroadcastHashJoin Inner BuildRight (32) + : : :- * Project (26) + : : : +- * BroadcastHashJoin Inner BuildRight (25) + : : : :- * Project (20) + : : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * ColumnarToRow (7) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- CometProject (13) + : : : : : +- CometFilter (12) + : : : : : +- CometScan parquet spark_catalog.default.promotion (11) + : : : : +- ReusedExchange (18) + : : : +- BroadcastExchange (24) + : : : +- * ColumnarToRow (23) + : : : +- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.customer (21) + : : +- BroadcastExchange (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.customer_address (27) + : +- BroadcastExchange (38) + : +- * ColumnarToRow (37) + : +- CometProject (36) + : +- CometFilter (35) + : +- CometScan parquet spark_catalog.default.item (34) + +- BroadcastExchange (65) + +- * HashAggregate (64) + +- Exchange (63) + +- * HashAggregate (62) + +- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * Project (58) + : +- * BroadcastHashJoin Inner BuildRight (57) + : :- * Project (55) + : : +- * BroadcastHashJoin Inner BuildRight (54) + : : :- * Project (52) + : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : :- * Project (49) + : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : :- * ColumnarToRow (46) + : : : : : +- CometFilter (45) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (44) + : : : : +- ReusedExchange (47) + : : : +- ReusedExchange (50) + : : +- ReusedExchange (53) + : +- ReusedExchange (56) + +- ReusedExchange (59) (unknown) Scan parquet spark_catalog.default.store_sales @@ -151,7 +147,7 @@ Join condition: None Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] -(18) ReusedExchange [Reuses operator id: 76] +(18) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#14] (19) BroadcastHashJoin [codegen id : 7] @@ -263,17 +259,11 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#21] Results [1]: [sum#22] -(42) RowToColumnar +(42) Exchange Input [1]: [sum#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(43) CometColumnarExchange -Input [1]: [sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) ColumnarToRow [codegen id : 15] -Input [1]: [sum#22] - -(45) HashAggregate [codegen id : 15] +(43) HashAggregate [codegen id : 15] Input [1]: [sum#22] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] @@ -288,122 +278,116 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(47) CometFilter +(45) CometFilter Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) -(48) ColumnarToRow [codegen id : 13] +(46) ColumnarToRow [codegen id : 13] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -(49) ReusedExchange [Reuses operator id: 8] +(47) ReusedExchange [Reuses operator id: 8] Output [1]: [s_store_sk#31] -(50) BroadcastHashJoin [codegen id : 13] +(48) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_store_sk#27] Right keys [1]: [s_store_sk#31] Join type: Inner Join condition: None -(51) Project [codegen id : 13] +(49) Project [codegen id : 13] Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] -(52) ReusedExchange [Reuses operator id: 76] +(50) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#32] -(53) BroadcastHashJoin [codegen id : 13] +(51) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_sold_date_sk#29] Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None -(54) Project [codegen id : 13] +(52) Project [codegen id : 13] Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] -(55) ReusedExchange [Reuses operator id: 24] +(53) ReusedExchange [Reuses operator id: 24] Output [2]: [c_customer_sk#33, c_current_addr_sk#34] -(56) BroadcastHashJoin [codegen id : 13] +(54) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#33] Join type: Inner Join condition: None -(57) Project [codegen id : 13] +(55) Project [codegen id : 13] Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] -(58) ReusedExchange [Reuses operator id: 31] +(56) ReusedExchange [Reuses operator id: 31] Output [1]: [ca_address_sk#35] -(59) BroadcastHashJoin [codegen id : 13] +(57) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#34] Right keys [1]: [ca_address_sk#35] Join type: Inner Join condition: None -(60) Project [codegen id : 13] +(58) Project [codegen id : 13] Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] -(61) ReusedExchange [Reuses operator id: 38] +(59) ReusedExchange [Reuses operator id: 38] Output [1]: [i_item_sk#36] -(62) BroadcastHashJoin [codegen id : 13] +(60) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_item_sk#25] Right keys [1]: [i_item_sk#36] Join type: Inner Join condition: None -(63) Project [codegen id : 13] +(61) Project [codegen id : 13] Output [1]: [ss_ext_sales_price#28] Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] -(64) HashAggregate [codegen id : 13] +(62) HashAggregate [codegen id : 13] Input [1]: [ss_ext_sales_price#28] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] Aggregate Attributes [1]: [sum#37] Results [1]: [sum#38] -(65) RowToColumnar -Input [1]: [sum#38] - -(66) CometColumnarExchange -Input [1]: [sum#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(67) ColumnarToRow [codegen id : 14] +(63) Exchange Input [1]: [sum#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(68) HashAggregate [codegen id : 14] +(64) HashAggregate [codegen id : 14] Input [1]: [sum#38] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#28))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#39,17,2) AS total#40] -(69) BroadcastExchange +(65) BroadcastExchange Input [1]: [total#40] Arguments: IdentityBroadcastMode, [plan_id=8] -(70) BroadcastNestedLoopJoin [codegen id : 15] +(66) BroadcastNestedLoopJoin [codegen id : 15] Join type: Inner Join condition: None -(71) Project [codegen id : 15] +(67) Project [codegen id : 15] Output [3]: [promotions#24, total#40, ((cast(promotions#24 as decimal(15,4)) / cast(total#40 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#41] Input [2]: [promotions#24, total#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) (unknown) Scan parquet spark_catalog.default.date_dim @@ -413,21 +397,21 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(69) CometFilter Input [3]: [d_date_sk#14, d_year#42, d_moy#43] Condition : ((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 1998)) AND (d_moy#43 = 11)) AND isnotnull(d_date_sk#14)) -(74) CometProject +(70) CometProject Input [3]: [d_date_sk#14, d_year#42, d_moy#43] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(76) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 46 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 44 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index a0df279d3b..2c3d07ac64 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -2,109 +2,105 @@ WholeStageCodegen (15) Project [promotions,total] BroadcastNestedLoopJoin HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] + InputAdapter + Exchange #1 + WholeStageCodegen (7) + HashAggregate [ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometProject [s_store_sk] + CometFilter [s_gmt_offset,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #5 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) + BroadcastExchange #6 + WholeStageCodegen (5) ColumnarToRow InputAdapter - CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #8 WholeStageCodegen (14) HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange #9 - RowToColumnar - WholeStageCodegen (13) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [s_store_sk] #3 + InputAdapter + Exchange #9 + WholeStageCodegen (13) + HashAggregate [ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [s_store_sk] #3 InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk] #5 + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [ca_address_sk] #6 + ReusedExchange [c_customer_sk,c_current_addr_sk] #5 InputAdapter - ReusedExchange [i_item_sk] #7 + ReusedExchange [ca_address_sk] #6 + InputAdapter + ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt index 602ce66e9a..82ded1c507 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt @@ -1,38 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * HashAggregate (33) - +- * ColumnarToRow (32) - +- CometColumnarExchange (31) - +- RowToColumnar (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.ship_mode (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.web_site (16) - +- BroadcastExchange (26) - +- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +TakeOrderedAndProject (32) ++- * HashAggregate (31) + +- Exchange (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.warehouse (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.ship_mode (10) + : +- BroadcastExchange (19) + : +- * ColumnarToRow (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.web_site (16) + +- BroadcastExchange (26) + +- * ColumnarToRow (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) (unknown) Scan parquet spark_catalog.default.web_sales @@ -172,24 +170,18 @@ Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) < Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] Results [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -(30) RowToColumnar +(30) Exchange Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, web_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(31) CometColumnarExchange -Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, web_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(32) ColumnarToRow [codegen id : 6] -Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] - -(33) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 6] Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [3]: [_groupingexpression#14, sm_type#9, web_name#11] Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] Aggregate Attributes [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] -(34) TakeOrderedAndProject +(32) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt index 7f7bee1305..5ae522ce1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt @@ -1,50 +1,48 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] WholeStageCodegen (6) HashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,sm_type,web_name] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,w_warehouse_name] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - Project [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] - CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + Exchange [_groupingexpression,sm_type,web_name] #1 + WholeStageCodegen (5) + HashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,w_warehouse_name] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Project [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] + CometFilter [sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index 76b5cfd50f..458a35b388 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -1,36 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * Filter (30) - +- Window (29) - +- * ColumnarToRow (28) - +- CometSort (27) - +- CometColumnarExchange (26) - +- RowToColumnar (25) - +- * HashAggregate (24) - +- * ColumnarToRow (23) - +- CometColumnarExchange (22) - +- RowToColumnar (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.store (14) +TakeOrderedAndProject (28) ++- * Project (27) + +- * Filter (26) + +- Window (25) + +- * Sort (24) + +- Exchange (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.store_sales (5) + : +- ReusedExchange (11) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) (unknown) Scan parquet spark_catalog.default.item @@ -80,7 +76,7 @@ Join condition: None Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -(11) ReusedExchange [Reuses operator id: 37] +(11) ReusedExchange [Reuses operator id: 33] Output [2]: [d_date_sk#15, d_moy#16] (12) BroadcastHashJoin [codegen id : 4] @@ -128,61 +124,49 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] Aggregate Attributes [1]: [sum#18] Results [3]: [i_manager_id#5, d_moy#16, sum#19] -(21) RowToColumnar +(21) Exchange Input [3]: [i_manager_id#5, d_moy#16, sum#19] +Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) CometColumnarExchange -Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) ColumnarToRow [codegen id : 5] -Input [3]: [i_manager_id#5, d_moy#16, sum#19] - -(24) HashAggregate [codegen id : 5] +(22) HashAggregate [codegen id : 5] Input [3]: [i_manager_id#5, d_moy#16, sum#19] Keys [2]: [i_manager_id#5, d_moy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] -(25) RowToColumnar -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] - -(26) CometColumnarExchange -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort +(23) Exchange Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] +Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) ColumnarToRow [codegen id : 6] +(24) Sort [codegen id : 6] Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +Arguments: [i_manager_id#5 ASC NULLS FIRST], false, 0 -(29) Window +(25) Window Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] -(30) Filter [codegen id : 7] +(26) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END -(31) Project [codegen id : 7] +(27) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -(32) TakeOrderedAndProject +(28) TakeOrderedAndProject Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) (unknown) Scan parquet spark_catalog.default.date_dim @@ -192,18 +176,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(30) CometFilter Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(35) CometProject +(31) CometProject Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] -(36) ColumnarToRow [codegen id : 1] +(32) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#16] -(37) BroadcastExchange +(33) BroadcastExchange Input [2]: [d_date_sk#15, d_moy#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index 1d50a8d34f..7f6f8c1370 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -5,51 +5,47 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] WholeStageCodegen (6) - ColumnarToRow + Sort [i_manager_id] InputAdapter - CometSort [i_manager_id] - CometColumnarExchange [i_manager_id] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_manager_id,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - Project [i_manager_id,ss_sales_price,d_moy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 + Exchange [i_manager_id] #1 + WholeStageCodegen (5) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_manager_id,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + Project [i_manager_id,ss_sales_price,d_moy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_manager_id] + CometFilter [i_category,i_class,i_brand,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index c4cbdaa3b5..a6f9571623 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -1,189 +1,185 @@ == Physical Plan == -* ColumnarToRow (185) -+- CometSort (184) - +- CometColumnarExchange (183) - +- RowToColumnar (182) - +- * Project (181) - +- * SortMergeJoin Inner (180) - :- * ColumnarToRow (110) - : +- CometSort (109) - : +- CometColumnarExchange (108) - : +- RowToColumnar (107) - : +- * HashAggregate (106) - : +- * HashAggregate (105) - : +- * Project (104) - : +- * BroadcastHashJoin Inner BuildRight (103) - : :- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * Project (94) - : : : +- * BroadcastHashJoin Inner BuildRight (93) - : : : :- * Project (88) - : : : : +- * BroadcastHashJoin Inner BuildRight (87) - : : : : :- * Project (85) - : : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : : :- * Project (79) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : :- * Project (76) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : : : : : :- * Project (70) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (69) - : : : : : : : : :- * Project (64) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : : : : : : :- * Project (61) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : : : : : : : :- * Project (55) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : : : : : : : :- * Project (52) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : : : : : : : : : : :- * Project (46) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : : : : : : : : : : :- * Project (40) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : : : : : : : : : : : : :- * Project (34) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : : : : : : : : : : : :- * ColumnarToRow (31) - : : : : : : : : : : : : : : : : +- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- ReusedExchange (32) - : : : : : : : : : : : : : : +- BroadcastExchange (38) - : : : : : : : : : : : : : : +- * ColumnarToRow (37) - : : : : : : : : : : : : : : +- CometFilter (36) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (35) - : : : : : : : : : : : : : +- BroadcastExchange (44) - : : : : : : : : : : : : : +- * ColumnarToRow (43) - : : : : : : : : : : : : : +- CometFilter (42) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) - : : : : : : : : : : : : +- BroadcastExchange (50) - : : : : : : : : : : : : +- * ColumnarToRow (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (53) - : : : : : : : : : : +- BroadcastExchange (59) - : : : : : : : : : : +- * ColumnarToRow (58) - : : : : : : : : : : +- CometFilter (57) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (56) - : : : : : : : : : +- ReusedExchange (62) - : : : : : : : : +- BroadcastExchange (68) - : : : : : : : : +- * ColumnarToRow (67) - : : : : : : : : +- CometFilter (66) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (65) - : : : : : : : +- BroadcastExchange (74) - : : : : : : : +- * ColumnarToRow (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (77) - : : : : : +- BroadcastExchange (83) - : : : : : +- * ColumnarToRow (82) - : : : : : +- CometFilter (81) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (80) - : : : : +- ReusedExchange (86) - : : : +- BroadcastExchange (92) - : : : +- * ColumnarToRow (91) - : : : +- CometFilter (90) - : : : +- CometScan parquet spark_catalog.default.income_band (89) - : : +- ReusedExchange (95) - : +- BroadcastExchange (102) - : +- * ColumnarToRow (101) - : +- CometProject (100) - : +- CometFilter (99) - : +- CometScan parquet spark_catalog.default.item (98) - +- * ColumnarToRow (179) - +- CometSort (178) - +- CometColumnarExchange (177) - +- RowToColumnar (176) - +- * HashAggregate (175) - +- * HashAggregate (174) - +- * Project (173) - +- * BroadcastHashJoin Inner BuildRight (172) - :- * Project (170) - : +- * BroadcastHashJoin Inner BuildRight (169) - : :- * Project (167) - : : +- * BroadcastHashJoin Inner BuildRight (166) - : : :- * Project (164) - : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : :- * Project (161) - : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : :- * Project (158) - : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : :- * Project (155) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : :- * Project (152) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : :- * Project (149) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) - : : : : : : : : :- * Project (146) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) - : : : : : : : : : :- * Project (143) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : : : : : : : : :- * Project (140) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : : : : : : : : :- * Project (137) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) - : : : : : : : : : : : : : : :- * ColumnarToRow (128) - : : : : : : : : : : : : : : : +- CometProject (127) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (126) - : : : : : : : : : : : : : : : :- CometSort (120) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (119) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) - : : : : : : : : : : : : : : : : : +- CometFilter (112) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometFilter (115) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) - : : : : : : : : : : : : : : : +- CometSort (125) - : : : : : : : : : : : : : : : +- CometProject (124) - : : : : : : : : : : : : : : : +- CometFilter (123) - : : : : : : : : : : : : : : : +- CometHashAggregate (122) - : : : : : : : : : : : : : : : +- ReusedExchange (121) - : : : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : +- ReusedExchange (147) - : : : : : : : +- ReusedExchange (150) - : : : : : : +- ReusedExchange (153) - : : : : : +- ReusedExchange (156) - : : : : +- ReusedExchange (159) - : : : +- ReusedExchange (162) - : : +- ReusedExchange (165) - : +- ReusedExchange (168) - +- ReusedExchange (171) +* Sort (181) ++- Exchange (180) + +- * Project (179) + +- * SortMergeJoin Inner (178) + :- * Sort (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- * Project (106) + : +- * BroadcastHashJoin Inner BuildRight (105) + : :- * Project (99) + : : +- * BroadcastHashJoin Inner BuildRight (98) + : : :- * Project (96) + : : : +- * BroadcastHashJoin Inner BuildRight (95) + : : : :- * Project (90) + : : : : +- * BroadcastHashJoin Inner BuildRight (89) + : : : : :- * Project (87) + : : : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : : : :- * Project (81) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) + : : : : : : :- * Project (78) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) + : : : : : : : :- * Project (72) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) + : : : : : : : : :- * Project (66) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) + : : : : : : : : : :- * Project (63) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : : : : : : : : :- * Project (57) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : : : : : : : : : :- * Project (54) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) + : : : : : : : : : : : : :- * Project (48) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : : : : : : : : : : :- * Project (42) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : : : : : : : : : : : :- * Project (36) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : : : : : : : : : : : :- * Project (33) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) + : : : : : : : : : : : : : : : : :- * Sort (11) + : : : : : : : : : : : : : : : : : +- Exchange (10) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- * Sort (31) + : : : : : : : : : : : : : : : : +- * Project (30) + : : : : : : : : : : : : : : : : +- * Filter (29) + : : : : : : : : : : : : : : : : +- * HashAggregate (28) + : : : : : : : : : : : : : : : : +- Exchange (27) + : : : : : : : : : : : : : : : : +- * HashAggregate (26) + : : : : : : : : : : : : : : : : +- * Project (25) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) + : : : : : : : : : : : : : : : : :- * Sort (17) + : : : : : : : : : : : : : : : : : +- Exchange (16) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) + : : : : : : : : : : : : : : : : : +- CometProject (14) + : : : : : : : : : : : : : : : : : +- CometFilter (13) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : : : : : : : : : : : : : : : +- * Sort (23) + : : : : : : : : : : : : : : : : +- Exchange (22) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) + : : : : : : : : : : : : : : : : +- CometProject (20) + : : : : : : : : : : : : : : : : +- CometFilter (19) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) + : : : : : : : : : : : : : : : +- ReusedExchange (34) + : : : : : : : : : : : : : : +- BroadcastExchange (40) + : : : : : : : : : : : : : : +- * ColumnarToRow (39) + : : : : : : : : : : : : : : +- CometFilter (38) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) + : : : : : : : : : : : : : +- BroadcastExchange (46) + : : : : : : : : : : : : : +- * ColumnarToRow (45) + : : : : : : : : : : : : : +- CometFilter (44) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) + : : : : : : : : : : : : +- BroadcastExchange (52) + : : : : : : : : : : : : +- * ColumnarToRow (51) + : : : : : : : : : : : : +- CometFilter (50) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) + : : : : : : : : : : : +- ReusedExchange (55) + : : : : : : : : : : +- BroadcastExchange (61) + : : : : : : : : : : +- * ColumnarToRow (60) + : : : : : : : : : : +- CometFilter (59) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) + : : : : : : : : : +- ReusedExchange (64) + : : : : : : : : +- BroadcastExchange (70) + : : : : : : : : +- * ColumnarToRow (69) + : : : : : : : : +- CometFilter (68) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) + : : : : : : : +- BroadcastExchange (76) + : : : : : : : +- * ColumnarToRow (75) + : : : : : : : +- CometFilter (74) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) + : : : : : : +- ReusedExchange (79) + : : : : : +- BroadcastExchange (85) + : : : : : +- * ColumnarToRow (84) + : : : : : +- CometFilter (83) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) + : : : : +- ReusedExchange (88) + : : : +- BroadcastExchange (94) + : : : +- * ColumnarToRow (93) + : : : +- CometFilter (92) + : : : +- CometScan parquet spark_catalog.default.income_band (91) + : : +- ReusedExchange (97) + : +- BroadcastExchange (104) + : +- * ColumnarToRow (103) + : +- CometProject (102) + : +- CometFilter (101) + : +- CometScan parquet spark_catalog.default.item (100) + +- * Sort (177) + +- Exchange (176) + +- * HashAggregate (175) + +- * HashAggregate (174) + +- * Project (173) + +- * BroadcastHashJoin Inner BuildRight (172) + :- * Project (170) + : +- * BroadcastHashJoin Inner BuildRight (169) + : :- * Project (167) + : : +- * BroadcastHashJoin Inner BuildRight (166) + : : :- * Project (164) + : : : +- * BroadcastHashJoin Inner BuildRight (163) + : : : :- * Project (161) + : : : : +- * BroadcastHashJoin Inner BuildRight (160) + : : : : :- * Project (158) + : : : : : +- * BroadcastHashJoin Inner BuildRight (157) + : : : : : :- * Project (155) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) + : : : : : : :- * Project (152) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) + : : : : : : : :- * Project (149) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) + : : : : : : : : :- * Project (146) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) + : : : : : : : : : :- * Project (143) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) + : : : : : : : : : : :- * Project (140) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) + : : : : : : : : : : : :- * Project (137) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) + : : : : : : : : : : : : :- * Project (134) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) + : : : : : : : : : : : : : :- * Project (131) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) + : : : : : : : : : : : : : : :- * Project (128) + : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) + : : : : : : : : : : : : : : : :- * Sort (121) + : : : : : : : : : : : : : : : : +- Exchange (120) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) + : : : : : : : : : : : : : : : : +- CometProject (118) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) + : : : : : : : : : : : : : : : : : +- CometFilter (112) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) + : : : : : : : : : : : : : : : : +- CometProject (116) + : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) + : : : : : : : : : : : : : : : +- * Sort (126) + : : : : : : : : : : : : : : : +- * Project (125) + : : : : : : : : : : : : : : : +- * Filter (124) + : : : : : : : : : : : : : : : +- * HashAggregate (123) + : : : : : : : : : : : : : : : +- ReusedExchange (122) + : : : : : : : : : : : : : : +- ReusedExchange (129) + : : : : : : : : : : : : : +- ReusedExchange (132) + : : : : : : : : : : : : +- ReusedExchange (135) + : : : : : : : : : : : +- ReusedExchange (138) + : : : : : : : : : : +- ReusedExchange (141) + : : : : : : : : : +- ReusedExchange (144) + : : : : : : : : +- ReusedExchange (147) + : : : : : : : +- ReusedExchange (150) + : : : : : : +- ReusedExchange (153) + : : : : : +- ReusedExchange (156) + : : : : +- ReusedExchange (159) + : : : +- ReusedExchange (162) + : : +- ReusedExchange (165) + : +- ReusedExchange (168) + +- ReusedExchange (171) (unknown) Scan parquet spark_catalog.default.store_sales @@ -226,13 +222,16 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) CometColumnarExchange +(9) ColumnarToRow [codegen id : 1] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) CometSort +(10) Exchange Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(11) Sort [codegen id : 2] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] @@ -241,21 +240,24 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(12) CometFilter +(13) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(13) CometProject +(14) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(14) CometColumnarExchange +(15) ColumnarToRow [codegen id : 3] +Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] + +(16) Exchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometSort +(17) Sort [codegen id : 4] Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] +Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] @@ -264,808 +266,799 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(17) CometFilter +(19) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(18) CometProject +(20) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(19) CometColumnarExchange +(21) ColumnarToRow [codegen id : 5] Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) CometSort +(22) Exchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner +(23) Sort [codegen id : 6] +Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 -(22) CometProject +(24) SortMergeJoin [codegen id : 7] +Left keys [2]: [cs_item_sk#17, cs_order_number#18] +Right keys [2]: [cr_item_sk#21, cr_order_number#22] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 7] +Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(23) CometHashAggregate +(26) HashAggregate [codegen id : 7] Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] +Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(24) CometColumnarExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(27) Exchange +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] +(28) HashAggregate [codegen id : 8] +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] +Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) +(29) Filter [codegen id : 8] +Input [3]: [cs_item_sk#17, sale#35, refund#36] +Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] +(30) Project [codegen id : 8] +Output [1]: [cs_item_sk#17] +Input [3]: [cs_item_sk#17, sale#35, refund#36] -(28) CometSort +(31) Sort [codegen id : 8] Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] +Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner +(32) SortMergeJoin [codegen id : 24] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#17] +Join type: Inner +Join condition: None -(30) CometProject +(33) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) ColumnarToRow [codegen id : 16] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(32) ReusedExchange [Reuses operator id: 189] -Output [2]: [d_date_sk#32, d_year#33] +(34) ReusedExchange [Reuses operator id: 185] +Output [2]: [d_date_sk#37, d_year#38] -(33) BroadcastHashJoin [codegen id : 16] +(35) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#32] +Right keys [1]: [d_date_sk#37] Join type: Inner Join condition: None -(34) Project [codegen id : 16] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] +(36) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] (unknown) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(36) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(s_zip#36)) +(38) CometFilter +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) -(37) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +(39) ColumnarToRow [codegen id : 10] +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -(38) BroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +(40) BroadcastExchange +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 16] +(41) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#34] +Right keys [1]: [s_store_sk#39] Join type: Inner Join condition: None -(40) Project [codegen id : 16] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#36] +(42) Project [codegen id : 24] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] (unknown) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(42) CometFilter -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -Condition : (((((isnotnull(c_customer_sk#37) AND isnotnull(c_first_sales_date_sk#42)) AND isnotnull(c_first_shipto_date_sk#41)) AND isnotnull(c_current_cdemo_sk#38)) AND isnotnull(c_current_hdemo_sk#39)) AND isnotnull(c_current_addr_sk#40)) +(44) CometFilter +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) -(43) ColumnarToRow [codegen id : 3] -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(45) ColumnarToRow [codegen id : 11] +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -(44) BroadcastExchange -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(46) BroadcastExchange +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 16] +(47) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#37] +Right keys [1]: [c_customer_sk#42] Join type: Inner Join condition: None -(46) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(48) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#48, d_year#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : isnotnull(d_date_sk#43) +(50) CometFilter +Input [2]: [d_date_sk#48, d_year#49] +Condition : isnotnull(d_date_sk#48) -(49) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#43, d_year#44] +(51) ColumnarToRow [codegen id : 12] +Input [2]: [d_date_sk#48, d_year#49] -(50) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] +(52) BroadcastExchange +Input [2]: [d_date_sk#48, d_year#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_first_sales_date_sk#42] -Right keys [1]: [d_date_sk#43] +(53) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_sales_date_sk#47] +Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42, d_date_sk#43, d_year#44] +(54) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] -(53) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#45, d_year#46] +(55) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#50, d_year#51] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_first_shipto_date_sk#41] -Right keys [1]: [d_date_sk#45] +(56) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_shipto_date_sk#46] +Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(55) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44, d_date_sk#45, d_year#46] +(57) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] (unknown) Scan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#47, cd_marital_status#48] +Output [2]: [cd_demo_sk#52, cd_marital_status#53] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(57) CometFilter -Input [2]: [cd_demo_sk#47, cd_marital_status#48] -Condition : (isnotnull(cd_demo_sk#47) AND isnotnull(cd_marital_status#48)) +(59) CometFilter +Input [2]: [cd_demo_sk#52, cd_marital_status#53] +Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) -(58) ColumnarToRow [codegen id : 6] -Input [2]: [cd_demo_sk#47, cd_marital_status#48] +(60) ColumnarToRow [codegen id : 14] +Input [2]: [cd_demo_sk#52, cd_marital_status#53] -(59) BroadcastExchange -Input [2]: [cd_demo_sk#47, cd_marital_status#48] +(61) BroadcastExchange +Input [2]: [cd_demo_sk#52, cd_marital_status#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 16] +(62) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#47] +Right keys [1]: [cd_demo_sk#52] Join type: Inner Join condition: None -(61) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_demo_sk#47, cd_marital_status#48] +(63) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] -(62) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#49, cd_marital_status#50] +(64) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#54, cd_marital_status#55] -(63) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_cdemo_sk#38] -Right keys [1]: [cd_demo_sk#49] +(65) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_cdemo_sk#43] +Right keys [1]: [cd_demo_sk#54] Join type: Inner -Join condition: NOT (cd_marital_status#48 = cd_marital_status#50) +Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) -(64) Project [codegen id : 16] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48, cd_demo_sk#49, cd_marital_status#50] +(66) Project [codegen id : 24] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] (unknown) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#51] +Output [1]: [p_promo_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(66) CometFilter -Input [1]: [p_promo_sk#51] -Condition : isnotnull(p_promo_sk#51) +(68) CometFilter +Input [1]: [p_promo_sk#56] +Condition : isnotnull(p_promo_sk#56) -(67) ColumnarToRow [codegen id : 8] -Input [1]: [p_promo_sk#51] +(69) ColumnarToRow [codegen id : 16] +Input [1]: [p_promo_sk#56] -(68) BroadcastExchange -Input [1]: [p_promo_sk#51] +(70) BroadcastExchange +Input [1]: [p_promo_sk#56] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(69) BroadcastHashJoin [codegen id : 16] +(71) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#51] +Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(70) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, p_promo_sk#51] +(72) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] (unknown) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(72) CometFilter -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] -Condition : (isnotnull(hd_demo_sk#52) AND isnotnull(hd_income_band_sk#53)) +(74) CometFilter +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) -(73) ColumnarToRow [codegen id : 9] -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +(75) ColumnarToRow [codegen id : 17] +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -(74) BroadcastExchange -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +(76) BroadcastExchange +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(75) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#52] +Right keys [1]: [hd_demo_sk#57] Join type: Inner Join condition: None -(76) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_demo_sk#52, hd_income_band_sk#53] +(78) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] -(77) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] +(79) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] -(78) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_hdemo_sk#39] -Right keys [1]: [hd_demo_sk#54] +(80) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_hdemo_sk#44] +Right keys [1]: [hd_demo_sk#59] Join type: Inner Join condition: None -(79) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_demo_sk#54, hd_income_band_sk#55] +(81) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] (unknown) Scan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(81) CometFilter -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -Condition : isnotnull(ca_address_sk#56) +(83) CometFilter +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Condition : isnotnull(ca_address_sk#61) -(82) ColumnarToRow [codegen id : 11] -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(84) ColumnarToRow [codegen id : 19] +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(83) BroadcastExchange -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(85) BroadcastExchange +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] -(84) BroadcastHashJoin [codegen id : 16] +(86) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#56] +Right keys [1]: [ca_address_sk#61] Join type: Inner Join condition: None -(85) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(87) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(86) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(88) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -(87) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#61] +(89) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_addr_sk#45] +Right keys [1]: [ca_address_sk#66] Join type: Inner Join condition: None -(88) Project [codegen id : 16] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(90) Project [codegen id : 24] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] (unknown) Scan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#66] +Output [1]: [ib_income_band_sk#71] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(90) CometFilter -Input [1]: [ib_income_band_sk#66] -Condition : isnotnull(ib_income_band_sk#66) +(92) CometFilter +Input [1]: [ib_income_band_sk#71] +Condition : isnotnull(ib_income_band_sk#71) -(91) ColumnarToRow [codegen id : 13] -Input [1]: [ib_income_band_sk#66] +(93) ColumnarToRow [codegen id : 21] +Input [1]: [ib_income_band_sk#71] -(92) BroadcastExchange -Input [1]: [ib_income_band_sk#66] +(94) BroadcastExchange +Input [1]: [ib_income_band_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(93) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [hd_income_band_sk#53] -Right keys [1]: [ib_income_band_sk#66] +(95) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#58] +Right keys [1]: [ib_income_band_sk#71] Join type: Inner Join condition: None -(94) Project [codegen id : 16] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#66] +(96) Project [codegen id : 24] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] -(95) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#67] +(97) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#72] -(96) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [hd_income_band_sk#55] -Right keys [1]: [ib_income_band_sk#67] +(98) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#60] +Right keys [1]: [ib_income_band_sk#72] Join type: Inner Join condition: None -(97) Project [codegen id : 16] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#67] +(99) Project [codegen id : 24] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] (unknown) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(99) CometFilter -Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] -Condition : ((((((isnotnull(i_current_price#69) AND i_color#70 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#69 >= 64.00)) AND (i_current_price#69 <= 74.00)) AND (i_current_price#69 >= 65.00)) AND (i_current_price#69 <= 79.00)) AND isnotnull(i_item_sk#68)) +(101) CometFilter +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) -(100) CometProject -Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] -Arguments: [i_item_sk#68, i_product_name#71], [i_item_sk#68, i_product_name#71] +(102) CometProject +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] -(101) ColumnarToRow [codegen id : 15] -Input [2]: [i_item_sk#68, i_product_name#71] +(103) ColumnarToRow [codegen id : 23] +Input [2]: [i_item_sk#73, i_product_name#76] -(102) BroadcastExchange -Input [2]: [i_item_sk#68, i_product_name#71] +(104) BroadcastExchange +Input [2]: [i_item_sk#73, i_product_name#76] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -(103) BroadcastHashJoin [codegen id : 16] +(105) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#68] +Right keys [1]: [i_item_sk#73] Join type: Inner Join condition: None -(104) Project [codegen id : 16] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +(106) Project [codegen id : 24] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -(105) HashAggregate [codegen id : 16] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] +(107) HashAggregate [codegen id : 24] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#72, sum#73, sum#74, sum#75] -Results [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] +Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] +Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -(106) HashAggregate [codegen id : 16] -Input [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] -Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] +(108) HashAggregate [codegen id : 24] +Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#9))#81, sum(UnscaledValue(ss_list_price#10))#82, sum(UnscaledValue(ss_coupon_amt#11))#83] -Results [17]: [i_product_name#71 AS product_name#84, i_item_sk#68 AS item_sk#85, s_store_name#35 AS store_name#86, s_zip#36 AS store_zip#87, ca_street_number#57 AS b_street_number#88, ca_street_name#58 AS b_streen_name#89, ca_city#59 AS b_city#90, ca_zip#60 AS b_zip#91, ca_street_number#62 AS c_street_number#92, ca_street_name#63 AS c_street_name#93, ca_city#64 AS c_city#94, ca_zip#65 AS c_zip#95, d_year#33 AS syear#96, count(1)#80 AS cnt#97, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#81,17,2) AS s1#98, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#82,17,2) AS s2#99, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#83,17,2) AS s3#100] - -(107) RowToColumnar -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] +Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(108) CometColumnarExchange -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(109) Exchange +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(109) CometSort -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100], [item_sk#85 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, store_zip#87 ASC NULLS FIRST] - -(110) ColumnarToRow [codegen id : 17] -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] +(110) Sort [codegen id : 25] +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#112), dynamicpruningexpression(ss_sold_date_sk#112 IN dynamicpruning#113)] +PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (112) CometFilter -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Condition : (((((((isnotnull(ss_item_sk#101) AND isnotnull(ss_ticket_number#108)) AND isnotnull(ss_store_sk#106)) AND isnotnull(ss_customer_sk#102)) AND isnotnull(ss_cdemo_sk#103)) AND isnotnull(ss_promo_sk#107)) AND isnotnull(ss_hdemo_sk#104)) AND isnotnull(ss_addr_sk#105)) +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) (113) CometBroadcastExchange -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] (unknown) Scan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] +Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (115) CometFilter -Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] -Condition : (isnotnull(sr_item_sk#114) AND isnotnull(sr_ticket_number#115)) +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) (116) CometProject -Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] -Arguments: [sr_item_sk#114, sr_ticket_number#115], [sr_item_sk#114, sr_ticket_number#115] +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] (117) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Right output [2]: [sr_item_sk#114, sr_ticket_number#115] -Arguments: [ss_item_sk#101, ss_ticket_number#108], [sr_item_sk#114, sr_ticket_number#115], Inner +Left output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Right output [2]: [sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner (118) CometProject -Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, sr_item_sk#114, sr_ticket_number#115] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -(119) CometColumnarExchange -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(119) ColumnarToRow [codegen id : 26] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -(120) CometSort -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101 ASC NULLS FIRST] +(120) Exchange +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(121) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] +(121) Sort [codegen id : 27] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 -(122) CometHashAggregate -Input [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] -Keys [1]: [cs_item_sk#117] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#121)), sum(((cr_refunded_cash#122 + cr_reversed_charge#123) + cr_store_credit#124))] +(122) ReusedExchange [Reuses operator id: 27] +Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -(123) CometFilter -Input [3]: [cs_item_sk#117, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) +(123) HashAggregate [codegen id : 33] +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +Keys [1]: [cs_item_sk#122] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] +Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#35, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#36] -(124) CometProject -Input [3]: [cs_item_sk#117, sale#30, refund#31] -Arguments: [cs_item_sk#117], [cs_item_sk#117] +(124) Filter [codegen id : 33] +Input [3]: [cs_item_sk#122, sale#35, refund#36] +Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) -(125) CometSort -Input [1]: [cs_item_sk#117] -Arguments: [cs_item_sk#117], [cs_item_sk#117 ASC NULLS FIRST] +(125) Project [codegen id : 33] +Output [1]: [cs_item_sk#122] +Input [3]: [cs_item_sk#122, sale#35, refund#36] -(126) CometSortMergeJoin -Left output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Right output [1]: [cs_item_sk#117] -Arguments: [ss_item_sk#101], [cs_item_sk#117], Inner +(126) Sort [codegen id : 33] +Input [1]: [cs_item_sk#122] +Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 -(127) CometProject -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, cs_item_sk#117] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(127) SortMergeJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [cs_item_sk#122] +Join type: Inner +Join condition: None -(128) ColumnarToRow [codegen id : 33] -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(128) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] -(129) ReusedExchange [Reuses operator id: 193] -Output [2]: [d_date_sk#125, d_year#126] +(129) ReusedExchange [Reuses operator id: 189] +Output [2]: [d_date_sk#130, d_year#131] -(130) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_date_sk#112] -Right keys [1]: [d_date_sk#125] +(130) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_sold_date_sk#117] +Right keys [1]: [d_date_sk#130] Join type: Inner Join condition: None -(131) Project [codegen id : 33] -Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126] -Input [13]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, d_date_sk#125, d_year#126] +(131) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] +Input [13]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] -(132) ReusedExchange [Reuses operator id: 38] -Output [3]: [s_store_sk#127, s_store_name#128, s_zip#129] +(132) ReusedExchange [Reuses operator id: 40] +Output [3]: [s_store_sk#132, s_store_name#133, s_zip#134] -(133) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#106] -Right keys [1]: [s_store_sk#127] +(133) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_store_sk#111] +Right keys [1]: [s_store_sk#132] Join type: Inner Join condition: None -(134) Project [codegen id : 33] -Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129] -Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_sk#127, s_store_name#128, s_zip#129] +(134) Project [codegen id : 49] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#134] -(135) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] +(135) ReusedExchange [Reuses operator id: 46] +Output [6]: [c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -(136) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_customer_sk#102] -Right keys [1]: [c_customer_sk#130] +(136) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_customer_sk#107] +Right keys [1]: [c_customer_sk#135] Join type: Inner Join condition: None -(137) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -Input [18]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] +(137) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -(138) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#136, d_year#137] +(138) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#141, d_year#142] -(139) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_first_sales_date_sk#135] -Right keys [1]: [d_date_sk#136] +(139) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_sales_date_sk#140] +Right keys [1]: [d_date_sk#141] Join type: Inner Join condition: None -(140) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135, d_date_sk#136, d_year#137] +(140) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140, d_date_sk#141, d_year#142] -(141) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#138, d_year#139] +(141) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#143, d_year#144] -(142) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_first_shipto_date_sk#134] -Right keys [1]: [d_date_sk#138] +(142) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_shipto_date_sk#139] +Right keys [1]: [d_date_sk#143] Join type: Inner Join condition: None -(143) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137, d_date_sk#138, d_year#139] +(143) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142, d_date_sk#143, d_year#144] -(144) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#140, cd_marital_status#141] +(144) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#145, cd_marital_status#146] -(145) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_cdemo_sk#103] -Right keys [1]: [cd_demo_sk#140] +(145) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_cdemo_sk#108] +Right keys [1]: [cd_demo_sk#145] Join type: Inner Join condition: None -(146) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_demo_sk#140, cd_marital_status#141] +(146) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_demo_sk#145, cd_marital_status#146] -(147) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#142, cd_marital_status#143] +(147) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#147, cd_marital_status#148] -(148) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_cdemo_sk#131] -Right keys [1]: [cd_demo_sk#142] +(148) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_cdemo_sk#136] +Right keys [1]: [cd_demo_sk#147] Join type: Inner -Join condition: NOT (cd_marital_status#141 = cd_marital_status#143) +Join condition: NOT (cd_marital_status#146 = cd_marital_status#148) -(149) Project [codegen id : 33] -Output [14]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [18]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141, cd_demo_sk#142, cd_marital_status#143] +(149) Project [codegen id : 49] +Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146, cd_demo_sk#147, cd_marital_status#148] -(150) ReusedExchange [Reuses operator id: 68] -Output [1]: [p_promo_sk#144] +(150) ReusedExchange [Reuses operator id: 70] +Output [1]: [p_promo_sk#149] -(151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_promo_sk#107] -Right keys [1]: [p_promo_sk#144] +(151) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_promo_sk#112] +Right keys [1]: [p_promo_sk#149] Join type: Inner Join condition: None -(152) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, p_promo_sk#144] +(152) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, p_promo_sk#149] -(153) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#145, hd_income_band_sk#146] +(153) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#150, hd_income_band_sk#151] -(154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#104] -Right keys [1]: [hd_demo_sk#145] +(154) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_hdemo_sk#109] +Right keys [1]: [hd_demo_sk#150] Join type: Inner Join condition: None -(155) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146] -Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_demo_sk#145, hd_income_band_sk#146] +(155) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_demo_sk#150, hd_income_band_sk#151] -(156) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] +(156) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] -(157) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_hdemo_sk#132] -Right keys [1]: [hd_demo_sk#147] +(157) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_hdemo_sk#137] +Right keys [1]: [hd_demo_sk#152] Join type: Inner Join condition: None -(158) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148] -Input [15]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_demo_sk#147, hd_income_band_sk#148] +(158) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153] +Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_demo_sk#152, hd_income_band_sk#153] -(159) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] +(159) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(160) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_addr_sk#105] -Right keys [1]: [ca_address_sk#149] +(160) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_addr_sk#110] +Right keys [1]: [ca_address_sk#154] Join type: Inner Join condition: None -(161) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -Input [18]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] +(161) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(162) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(162) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -(163) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_addr_sk#133] -Right keys [1]: [ca_address_sk#154] +(163) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_addr_sk#138] +Right keys [1]: [ca_address_sk#159] Join type: Inner Join condition: None -(164) Project [codegen id : 33] -Output [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [21]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(164) Project [codegen id : 49] +Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -(165) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#159] +(165) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#164] -(166) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [hd_income_band_sk#146] -Right keys [1]: [ib_income_band_sk#159] +(166) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#151] +Right keys [1]: [ib_income_band_sk#164] Join type: Inner Join condition: None -(167) Project [codegen id : 33] -Output [18]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [20]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#159] +(167) Project [codegen id : 49] +Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#164] -(168) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#160] +(168) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#165] -(169) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [hd_income_band_sk#148] -Right keys [1]: [ib_income_band_sk#160] +(169) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#153] +Right keys [1]: [ib_income_band_sk#165] Join type: Inner Join condition: None -(170) Project [codegen id : 33] -Output [17]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#160] +(170) Project [codegen id : 49] +Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#165] -(171) ReusedExchange [Reuses operator id: 102] -Output [2]: [i_item_sk#161, i_product_name#162] +(171) ReusedExchange [Reuses operator id: 104] +Output [2]: [i_item_sk#166, i_product_name#167] -(172) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_item_sk#101] -Right keys [1]: [i_item_sk#161] +(172) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [i_item_sk#166] Join type: Inner Join condition: None -(173) Project [codegen id : 33] -Output [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] - -(174) HashAggregate [codegen id : 33] -Input [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#109)), partial_sum(UnscaledValue(ss_list_price#110)), partial_sum(UnscaledValue(ss_coupon_amt#111))] -Aggregate Attributes [4]: [count#72, sum#163, sum#164, sum#165] -Results [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] - -(175) HashAggregate [codegen id : 33] -Input [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] -Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#109)), sum(UnscaledValue(ss_list_price#110)), sum(UnscaledValue(ss_coupon_amt#111))] -Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#109))#81, sum(UnscaledValue(ss_list_price#110))#82, sum(UnscaledValue(ss_coupon_amt#111))#83] -Results [8]: [i_item_sk#161 AS item_sk#169, s_store_name#128 AS store_name#170, s_zip#129 AS store_zip#171, d_year#126 AS syear#172, count(1)#80 AS cnt#173, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#109))#81,17,2) AS s1#174, MakeDecimal(sum(UnscaledValue(ss_list_price#110))#82,17,2) AS s2#175, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#111))#83,17,2) AS s3#176] - -(176) RowToColumnar -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] - -(177) CometColumnarExchange -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(178) CometSort -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176], [item_sk#169 ASC NULLS FIRST, store_name#170 ASC NULLS FIRST, store_zip#171 ASC NULLS FIRST] - -(179) ColumnarToRow [codegen id : 34] -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] - -(180) SortMergeJoin [codegen id : 35] -Left keys [3]: [item_sk#85, store_name#86, store_zip#87] -Right keys [3]: [item_sk#169, store_name#170, store_zip#171] +(173) Project [codegen id : 49] +Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] + +(174) HashAggregate [codegen id : 49] +Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] +Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count#77, sum#168, sum#169, sum#170] +Results [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] + +(175) HashAggregate [codegen id : 49] +Input [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] +Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] +Results [8]: [i_item_sk#166 AS item_sk#174, s_store_name#133 AS store_name#175, s_zip#134 AS store_zip#176, d_year#131 AS syear#177, count(1)#85 AS cnt#178, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#179, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#180, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#181] + +(176) Exchange +Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +Arguments: hashpartitioning(item_sk#174, store_name#175, store_zip#176, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(177) Sort [codegen id : 50] +Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +Arguments: [item_sk#174 ASC NULLS FIRST, store_name#175 ASC NULLS FIRST, store_zip#176 ASC NULLS FIRST], false, 0 + +(178) SortMergeJoin [codegen id : 51] +Left keys [3]: [item_sk#90, store_name#91, store_zip#92] +Right keys [3]: [item_sk#174, store_name#175, store_zip#176] Join type: Inner -Join condition: (cnt#173 <= cnt#97) - -(181) Project [codegen id : 35] -Output [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Input [25]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] +Join condition: (cnt#178 <= cnt#102) -(182) RowToColumnar -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] +(179) Project [codegen id : 51] +Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -(183) CometColumnarExchange -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +(180) Exchange +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] -(184) CometSort -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173], [product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST] - -(185) ColumnarToRow [codegen id : 36] -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] +(181) Sort [codegen id : 52] +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (189) -+- * ColumnarToRow (188) - +- CometFilter (187) - +- CometScan parquet spark_catalog.default.date_dim (186) +BroadcastExchange (185) ++- * ColumnarToRow (184) + +- CometFilter (183) + +- CometScan parquet spark_catalog.default.date_dim (182) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] +Output [2]: [d_date_sk#37, d_year#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(187) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) +(183) CometFilter +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) -(188) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] +(184) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#37, d_year#38] -(189) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] +(185) BroadcastExchange +Input [2]: [d_date_sk#37, d_year#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 -BroadcastExchange (193) -+- * ColumnarToRow (192) - +- CometFilter (191) - +- CometScan parquet spark_catalog.default.date_dim (190) +Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (189) ++- * ColumnarToRow (188) + +- CometFilter (187) + +- CometScan parquet spark_catalog.default.date_dim (186) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#125, d_year#126] +Output [2]: [d_date_sk#130, d_year#131] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(191) CometFilter -Input [2]: [d_date_sk#125, d_year#126] -Condition : ((isnotnull(d_year#126) AND (d_year#126 = 2000)) AND isnotnull(d_date_sk#125)) +(187) CometFilter +Input [2]: [d_date_sk#130, d_year#131] +Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) -(192) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#125, d_year#126] +(188) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#130, d_year#131] -(193) BroadcastExchange -Input [2]: [d_date_sk#125, d_year#126] +(189) BroadcastExchange +Input [2]: [d_date_sk#130, d_year#131] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index 9fb039a96d..d972e0082c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -1,58 +1,58 @@ -WholeStageCodegen (36) - ColumnarToRow +WholeStageCodegen (52) + Sort [product_name,store_name,cnt] InputAdapter - CometSort [product_name,store_name,cnt] - CometColumnarExchange [product_name,store_name,cnt] #1 - RowToColumnar - WholeStageCodegen (35) - Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] - InputAdapter - WholeStageCodegen (17) - ColumnarToRow - InputAdapter - CometSort [item_sk,store_name,store_zip] - CometColumnarExchange [item_sk,store_name,store_zip] #2 - RowToColumnar - WholeStageCodegen (16) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow + Exchange [product_name,store_name,cnt] #1 + WholeStageCodegen (51) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (25) + Sort [item_sk,store_name,store_zip] + InputAdapter + Exchange [item_sk,store_name,store_zip] #2 + WholeStageCodegen (24) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometColumnarExchange [ss_item_sk] #3 + Exchange [ss_item_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] CometBroadcastExchange #4 @@ -68,146 +68,164 @@ WholeStageCodegen (36) CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - CometColumnarExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometColumnarExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number] - CometColumnarExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + WholeStageCodegen (8) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #6 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (4) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #8 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (4) + BroadcastExchange #9 + WholeStageCodegen (10) ColumnarToRow InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + BroadcastExchange #10 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (6) + BroadcastExchange #11 + WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) + BroadcastExchange #12 + WholeStageCodegen (14) ColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - BroadcastExchange #14 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + BroadcastExchange #13 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) + BroadcastExchange #14 + WholeStageCodegen (17) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (13) + BroadcastExchange #15 + WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter - ReusedExchange [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - BroadcastExchange #17 - WholeStageCodegen (15) + BroadcastExchange #16 + WholeStageCodegen (21) ColumnarToRow InputAdapter - CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - InputAdapter - WholeStageCodegen (34) - ColumnarToRow - InputAdapter - CometSort [item_sk,store_name,store_zip] - CometColumnarExchange [item_sk,store_name,store_zip] #18 - RowToColumnar - WholeStageCodegen (33) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (23) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_product_name] + CometFilter [i_current_price,i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (50) + Sort [item_sk,store_name,store_zip] + InputAdapter + Exchange [item_sk,store_name,store_zip] #18 + WholeStageCodegen (49) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (27) + Sort [ss_item_sk] InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometColumnarExchange [ss_item_sk] #19 + Exchange [ss_item_sk] #19 + WholeStageCodegen (26) + ColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] CometBroadcastExchange #20 @@ -223,38 +241,41 @@ WholeStageCodegen (36) CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #21 InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + WholeStageCodegen (33) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 + ReusedExchange [d_date_sk,d_year] #21 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [p_promo_sk] #13 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [p_promo_sk] #13 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [i_item_sk,i_product_name] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 007cf9bcd2..1a06f27fb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -1,49 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) +TakeOrderedAndProject (39) ++- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) : : :- * ColumnarToRow (3) : : : +- CometFilter (2) : : : +- CometScan parquet spark_catalog.default.store (1) - : : +- BroadcastExchange (16) - : : +- * Filter (15) - : : +- * HashAggregate (14) - : : +- * ColumnarToRow (13) - : : +- CometColumnarExchange (12) - : : +- RowToColumnar (11) - : : +- * HashAggregate (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : +- ReusedExchange (7) - : +- BroadcastExchange (22) - : +- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (42) - +- * Filter (41) - +- * HashAggregate (40) - +- * ColumnarToRow (39) - +- CometColumnarExchange (38) - +- RowToColumnar (37) - +- * HashAggregate (36) - +- * HashAggregate (35) - +- * ColumnarToRow (34) - +- CometColumnarExchange (33) - +- RowToColumnar (32) - +- * HashAggregate (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store_sales (25) - +- ReusedExchange (28) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * HashAggregate (12) + : : +- Exchange (11) + : : +- * HashAggregate (10) + : : +- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : +- ReusedExchange (7) + : +- BroadcastExchange (20) + : +- * ColumnarToRow (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + +- BroadcastExchange (36) + +- * Filter (35) + +- * HashAggregate (34) + +- Exchange (33) + +- * HashAggregate (32) + +- * HashAggregate (31) + +- Exchange (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * ColumnarToRow (25) + : +- CometFilter (24) + : +- CometScan parquet spark_catalog.default.store_sales (23) + +- ReusedExchange (26) (unknown) Scan parquet spark_catalog.default.store @@ -75,7 +69,7 @@ Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) (6) ColumnarToRow [codegen id : 2] Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -(7) ReusedExchange [Reuses operator id: 50] +(7) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#8] (8) BroadcastHashJoin [codegen id : 2] @@ -95,38 +89,32 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] Aggregate Attributes [1]: [sum#9] Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -(11) RowToColumnar +(11) Exchange Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] +Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(12) CometColumnarExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(13) ColumnarToRow [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] - -(14) HashAggregate [codegen id : 3] +(12) HashAggregate [codegen id : 3] Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] -(15) Filter [codegen id : 3] +(13) Filter [codegen id : 3] Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] Condition : isnotnull(revenue#12) -(16) BroadcastExchange +(14) BroadcastExchange Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(15) BroadcastHashJoin [codegen id : 9] Left keys [1]: [s_store_sk#1] Right keys [1]: [ss_store_sk#4] Join type: Inner Join condition: None -(18) Project [codegen id : 9] +(16) Project [codegen id : 9] Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] @@ -137,24 +125,24 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(20) CometFilter +(18) CometFilter Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Condition : isnotnull(i_item_sk#13) -(21) ColumnarToRow [codegen id : 4] +(19) ColumnarToRow [codegen id : 4] Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -(22) BroadcastExchange +(20) BroadcastExchange Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 9] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#3] Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(24) Project [codegen id : 9] +(22) Project [codegen id : 9] Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] @@ -166,104 +154,92 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(26) CometFilter +(24) CometFilter Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] Condition : isnotnull(ss_store_sk#19) -(27) ColumnarToRow [codegen id : 6] +(25) ColumnarToRow [codegen id : 6] Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -(28) ReusedExchange [Reuses operator id: 50] +(26) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#23] -(29) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#21] Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(30) Project [codegen id : 6] +(28) Project [codegen id : 6] Output [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] -(31) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] Keys [2]: [ss_store_sk#19, ss_item_sk#18] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] Aggregate Attributes [1]: [sum#24] Results [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] -(32) RowToColumnar -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] - -(33) CometColumnarExchange +(30) Exchange Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] -Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(34) ColumnarToRow [codegen id : 7] -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] - -(35) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 7] Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] Keys [2]: [ss_store_sk#19, ss_item_sk#18] Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#20))#26] Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#26,17,2) AS revenue#27] -(36) HashAggregate [codegen id : 7] +(32) HashAggregate [codegen id : 7] Input [2]: [ss_store_sk#19, revenue#27] Keys [1]: [ss_store_sk#19] Functions [1]: [partial_avg(revenue#27)] Aggregate Attributes [2]: [sum#28, count#29] Results [3]: [ss_store_sk#19, sum#30, count#31] -(37) RowToColumnar -Input [3]: [ss_store_sk#19, sum#30, count#31] - -(38) CometColumnarExchange -Input [3]: [ss_store_sk#19, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(39) ColumnarToRow [codegen id : 8] +(33) Exchange Input [3]: [ss_store_sk#19, sum#30, count#31] +Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(40) HashAggregate [codegen id : 8] +(34) HashAggregate [codegen id : 8] Input [3]: [ss_store_sk#19, sum#30, count#31] Keys [1]: [ss_store_sk#19] Functions [1]: [avg(revenue#27)] Aggregate Attributes [1]: [avg(revenue#27)#32] Results [2]: [ss_store_sk#19, avg(revenue#27)#32 AS ave#33] -(41) Filter [codegen id : 8] +(35) Filter [codegen id : 8] Input [2]: [ss_store_sk#19, ave#33] Condition : isnotnull(ave#33) -(42) BroadcastExchange +(36) BroadcastExchange Input [2]: [ss_store_sk#19, ave#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#19] Join type: Inner Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) -(44) Project [codegen id : 9] +(38) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17, ss_store_sk#19, ave#33] -(45) TakeOrderedAndProject +(39) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) (unknown) Scan parquet spark_catalog.default.date_dim @@ -273,21 +249,21 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(41) CometFilter Input [2]: [d_date_sk#8, d_month_seq#34] Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_month_seq#34 <= 1187)) AND isnotnull(d_date_sk#8)) -(48) CometProject +(42) CometProject Input [2]: [d_date_sk#8, d_month_seq#34] Arguments: [d_date_sk#8], [d_date_sk#8] -(49) ColumnarToRow [codegen id : 1] +(43) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(50) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index 880e68a392..33b695e811 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -15,28 +15,26 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (3) Filter [revenue] HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk,ss_item_sk] #2 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + Exchange [ss_store_sk,ss_item_sk] #2 + WholeStageCodegen (2) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 WholeStageCodegen (4) @@ -49,25 +47,21 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (8) Filter [ave] HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk] #6 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk,ss_item_sk] #7 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + Exchange [ss_store_sk] #6 + WholeStageCodegen (7) + HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] + InputAdapter + Exchange [ss_store_sk,ss_item_sk] #7 + WholeStageCodegen (6) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 0189e14e7c..41e783d134 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -1,62 +1,56 @@ == Physical Plan == -TakeOrderedAndProject (58) -+- * HashAggregate (57) - +- * ColumnarToRow (56) - +- CometColumnarExchange (55) - +- RowToColumnar (54) - +- * HashAggregate (53) - +- Union (52) - :- * HashAggregate (31) - : +- * ColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- RowToColumnar (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.time_dim (13) - : +- BroadcastExchange (24) - : +- * ColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.ship_mode (20) - +- * HashAggregate (51) - +- * ColumnarToRow (50) - +- CometColumnarExchange (49) - +- RowToColumnar (48) - +- * HashAggregate (47) - +- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (43) - : +- * BroadcastHashJoin Inner BuildRight (42) - : :- * Project (40) - : : +- * BroadcastHashJoin Inner BuildRight (39) - : : :- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * ColumnarToRow (34) - : : : : +- CometFilter (33) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (35) - : : +- ReusedExchange (38) - : +- ReusedExchange (41) - +- ReusedExchange (44) +TakeOrderedAndProject (52) ++- * HashAggregate (51) + +- Exchange (50) + +- * HashAggregate (49) + +- Union (48) + :- * HashAggregate (29) + : +- Exchange (28) + : +- * HashAggregate (27) + : +- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.warehouse (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (17) + : : +- * ColumnarToRow (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.time_dim (13) + : +- BroadcastExchange (24) + : +- * ColumnarToRow (23) + : +- CometProject (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.ship_mode (20) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (32) + : : : : +- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) + : : : +- ReusedExchange (33) + : : +- ReusedExchange (36) + : +- ReusedExchange (39) + +- ReusedExchange (42) (unknown) Scan parquet spark_catalog.default.web_sales @@ -102,7 +96,7 @@ Join condition: None Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -(10) ReusedExchange [Reuses operator id: 62] +(10) ReusedExchange [Reuses operator id: 56] Output [3]: [d_date_sk#16, d_year#17, d_moy#18] (11) BroadcastHashJoin [codegen id : 5] @@ -186,17 +180,11 @@ Functions [24]: [partial_sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 Aggregate Attributes [48]: [sum#23, isEmpty#24, sum#25, isEmpty#26, sum#27, isEmpty#28, sum#29, isEmpty#30, sum#31, isEmpty#32, sum#33, isEmpty#34, sum#35, isEmpty#36, sum#37, isEmpty#38, sum#39, isEmpty#40, sum#41, isEmpty#42, sum#43, isEmpty#44, sum#45, isEmpty#46, sum#47, isEmpty#48, sum#49, isEmpty#50, sum#51, isEmpty#52, sum#53, isEmpty#54, sum#55, isEmpty#56, sum#57, isEmpty#58, sum#59, isEmpty#60, sum#61, isEmpty#62, sum#63, isEmpty#64, sum#65, isEmpty#66, sum#67, isEmpty#68, sum#69, isEmpty#70] Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] -(28) RowToColumnar +(28) Exchange Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) CometColumnarExchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) ColumnarToRow [codegen id : 6] -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] - -(31) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17] Functions [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] @@ -211,126 +199,114 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#175), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(33) CometFilter +(31) CometFilter Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] Condition : ((isnotnull(cs_warehouse_sk#171) AND isnotnull(cs_sold_time_sk#169)) AND isnotnull(cs_ship_mode_sk#170)) -(34) ColumnarToRow [codegen id : 11] +(32) ColumnarToRow [codegen id : 11] Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] -(35) ReusedExchange [Reuses operator id: 7] +(33) ReusedExchange [Reuses operator id: 7] Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -(36) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_warehouse_sk#171] Right keys [1]: [w_warehouse_sk#177] Join type: Inner Join condition: None -(37) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [12]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -(38) ReusedExchange [Reuses operator id: 62] +(36) ReusedExchange [Reuses operator id: 56] Output [3]: [d_date_sk#184, d_year#185, d_moy#186] -(39) BroadcastHashJoin [codegen id : 11] +(37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#175] Right keys [1]: [d_date_sk#184] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(38) Project [codegen id : 11] Output [13]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [15]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] -(41) ReusedExchange [Reuses operator id: 17] +(39) ReusedExchange [Reuses operator id: 17] Output [1]: [t_time_sk#187] -(42) BroadcastHashJoin [codegen id : 11] +(40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_time_sk#169] Right keys [1]: [t_time_sk#187] Join type: Inner Join condition: None -(43) Project [codegen id : 11] +(41) Project [codegen id : 11] Output [12]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] -(44) ReusedExchange [Reuses operator id: 24] +(42) ReusedExchange [Reuses operator id: 24] Output [1]: [sm_ship_mode_sk#188] -(45) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_mode_sk#170] Right keys [1]: [sm_ship_mode_sk#188] Join type: Inner Join condition: None -(46) Project [codegen id : 11] +(44) Project [codegen id : 11] Output [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [13]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] -(47) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] Aggregate Attributes [48]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236] Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -(48) RowToColumnar -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] - -(49) CometColumnarExchange +(46) Exchange Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(50) ColumnarToRow [codegen id : 12] -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] - -(51) HashAggregate [codegen id : 12] +(47) HashAggregate [codegen id : 12] Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308] Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, DHL,BARIAN AS ship_carriers#309, d_year#185 AS year#310, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285 AS jan_sales#311, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286 AS feb_sales#312, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287 AS mar_sales#313, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288 AS apr_sales#314, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289 AS may_sales#315, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290 AS jun_sales#316, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291 AS jul_sales#317, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292 AS aug_sales#318, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293 AS sep_sales#319, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294 AS oct_sales#320, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295 AS nov_sales#321, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296 AS dec_sales#322, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297 AS jan_net#323, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298 AS feb_net#324, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299 AS mar_net#325, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300 AS apr_net#326, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301 AS may_net#327, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302 AS jun_net#328, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303 AS jul_net#329, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304 AS aug_net#330, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305 AS sep_net#331, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306 AS oct_net#332, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307 AS nov_net#333, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308 AS dec_net#334] -(52) Union +(48) Union -(53) HashAggregate [codegen id : 13] +(49) HashAggregate [codegen id : 13] Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#145, feb_sales#146, mar_sales#147, apr_sales#148, may_sales#149, jun_sales#150, jul_sales#151, aug_sales#152, sep_sales#153, oct_sales#154, nov_sales#155, dec_sales#156, jan_net#157, feb_net#158, mar_net#159, apr_net#160, may_net#161, jun_net#162, jul_net#163, aug_net#164, sep_net#165, oct_net#166, nov_net#167, dec_net#168] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial_sum(mar_sales#147), partial_sum(apr_sales#148), partial_sum(may_sales#149), partial_sum(jun_sales#150), partial_sum(jul_sales#151), partial_sum(aug_sales#152), partial_sum(sep_sales#153), partial_sum(oct_sales#154), partial_sum(nov_sales#155), partial_sum(dec_sales#156), partial_sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#157), partial_sum(feb_net#158), partial_sum(mar_net#159), partial_sum(apr_net#160), partial_sum(may_net#161), partial_sum(jun_net#162), partial_sum(jul_net#163), partial_sum(aug_net#164), partial_sum(sep_net#165), partial_sum(oct_net#166), partial_sum(nov_net#167), partial_sum(dec_net#168)] Aggregate Attributes [72]: [sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384, sum#385, isEmpty#386, sum#387, isEmpty#388, sum#389, isEmpty#390, sum#391, isEmpty#392, sum#393, isEmpty#394, sum#395, isEmpty#396, sum#397, isEmpty#398, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406] Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -(54) RowToColumnar -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] - -(55) CometColumnarExchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(56) ColumnarToRow [codegen id : 14] +(50) Exchange Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(57) HashAggregate [codegen id : 14] +(51) HashAggregate [codegen id : 14] Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#157), sum(feb_net#158), sum(mar_net#159), sum(apr_net#160), sum(may_net#161), sum(jun_net#162), sum(jul_net#163), sum(aug_net#164), sum(sep_net#165), sum(oct_net#166), sum(nov_net#167), sum(dec_net#168)] Aggregate Attributes [36]: [sum(jan_sales#145)#479, sum(feb_sales#146)#480, sum(mar_sales#147)#481, sum(apr_sales#148)#482, sum(may_sales#149)#483, sum(jun_sales#150)#484, sum(jul_sales#151)#485, sum(aug_sales#152)#486, sum(sep_sales#153)#487, sum(oct_sales#154)#488, sum(nov_sales#155)#489, sum(dec_sales#156)#490, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502, sum(jan_net#157)#503, sum(feb_net#158)#504, sum(mar_net#159)#505, sum(apr_net#160)#506, sum(may_net#161)#507, sum(jun_net#162)#508, sum(jul_net#163)#509, sum(aug_net#164)#510, sum(sep_net#165)#511, sum(oct_net#166)#512, sum(nov_net#167)#513, sum(dec_net#168)#514] Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum(jan_sales#145)#479 AS jan_sales#515, sum(feb_sales#146)#480 AS feb_sales#516, sum(mar_sales#147)#481 AS mar_sales#517, sum(apr_sales#148)#482 AS apr_sales#518, sum(may_sales#149)#483 AS may_sales#519, sum(jun_sales#150)#484 AS jun_sales#520, sum(jul_sales#151)#485 AS jul_sales#521, sum(aug_sales#152)#486 AS aug_sales#522, sum(sep_sales#153)#487 AS sep_sales#523, sum(oct_sales#154)#488 AS oct_sales#524, sum(nov_sales#155)#489 AS nov_sales#525, sum(dec_sales#156)#490 AS dec_sales#526, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491 AS jan_sales_per_sq_foot#527, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492 AS feb_sales_per_sq_foot#528, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493 AS mar_sales_per_sq_foot#529, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494 AS apr_sales_per_sq_foot#530, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495 AS may_sales_per_sq_foot#531, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496 AS jun_sales_per_sq_foot#532, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497 AS jul_sales_per_sq_foot#533, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498 AS aug_sales_per_sq_foot#534, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499 AS sep_sales_per_sq_foot#535, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500 AS oct_sales_per_sq_foot#536, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501 AS nov_sales_per_sq_foot#537, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502 AS dec_sales_per_sq_foot#538, sum(jan_net#157)#503 AS jan_net#539, sum(feb_net#158)#504 AS feb_net#540, sum(mar_net#159)#505 AS mar_net#541, sum(apr_net#160)#506 AS apr_net#542, sum(may_net#161)#507 AS may_net#543, sum(jun_net#162)#508 AS jun_net#544, sum(jul_net#163)#509 AS jul_net#545, sum(aug_net#164)#510 AS aug_net#546, sum(sep_net#165)#511 AS sep_net#547, sum(oct_net#166)#512 AS oct_net#548, sum(nov_net#167)#513 AS nov_net#549, sum(dec_net#168)#514 AS dec_net#550] -(58) TakeOrderedAndProject +(52) TakeOrderedAndProject Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (62) -+- * ColumnarToRow (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.date_dim (59) +BroadcastExchange (56) ++- * ColumnarToRow (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) (unknown) Scan parquet spark_catalog.default.date_dim @@ -340,17 +316,17 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(60) CometFilter +(54) CometFilter Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(61) ColumnarToRow [codegen id : 1] +(55) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -(62) BroadcastExchange +(56) BroadcastExchange Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#175 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#175 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index 3c5f5bd2f2..8ed74582f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -1,92 +1,86 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] WholeStageCodegen (14) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - RowToColumnar - WholeStageCodegen (13) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 + WholeStageCodegen (13) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 + WholeStageCodegen (5) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #4 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 + ReusedExchange [d_date_sk,d_year,d_moy] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_time,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [sm_ship_mode_sk] + CometFilter [sm_carrier,sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + WholeStageCodegen (12) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 + WholeStageCodegen (11) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_sold_time_sk,t_time_sk] + Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [t_time_sk] #5 + ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 InputAdapter - ReusedExchange [sm_ship_mode_sk] #6 + ReusedExchange [d_date_sk,d_year,d_moy] #3 + InputAdapter + ReusedExchange [t_time_sk] #5 + InputAdapter + ReusedExchange [sm_ship_mode_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index f84c06d92d..5dabc82d2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -1,35 +1,31 @@ == Physical Plan == -TakeOrderedAndProject (31) -+- * Filter (30) - +- Window (29) - +- * ColumnarToRow (28) - +- CometSort (27) - +- CometColumnarExchange (26) - +- RowToColumnar (25) - +- * HashAggregate (24) - +- * ColumnarToRow (23) - +- CometColumnarExchange (22) - +- RowToColumnar (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.item (13) +TakeOrderedAndProject (27) ++- * Filter (26) + +- Window (25) + +- * Sort (24) + +- Exchange (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Expand (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.store (7) + +- BroadcastExchange (16) + +- * ColumnarToRow (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.item (13) (unknown) Scan parquet spark_catalog.default.store_sales @@ -47,7 +43,7 @@ Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 36] +(4) ReusedExchange [Reuses operator id: 32] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (5) BroadcastHashJoin [codegen id : 4] @@ -127,57 +123,45 @@ Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as d Aggregate Attributes [2]: [sum#27, isEmpty#28] Results [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] -(21) RowToColumnar +(21) Exchange Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] +Arguments: hashpartitioning(i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) CometColumnarExchange -Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] -Arguments: hashpartitioning(i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) ColumnarToRow [codegen id : 5] -Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] - -(24) HashAggregate [codegen id : 5] +(22) HashAggregate [codegen id : 5] Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31] Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31 AS sumsales#32] -(25) RowToColumnar -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] - -(26) CometColumnarExchange -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort +(23) Exchange Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32], [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST] +Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) ColumnarToRow [codegen id : 6] +(24) Sort [codegen id : 6] Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] +Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0 -(29) Window +(25) Window Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] Arguments: [rank(sumsales#32) windowspecdefinition(i_category#18, sumsales#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#33], [i_category#18], [sumsales#32 DESC NULLS LAST] -(30) Filter [codegen id : 7] +(26) Filter [codegen id : 7] Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] Condition : (rk#33 <= 100) -(31) TakeOrderedAndProject +(27) TakeOrderedAndProject Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_qoy#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, sumsales#32 ASC NULLS FIRST, rk#33 ASC NULLS FIRST], [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (32) ++- * ColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan parquet spark_catalog.default.date_dim (28) (unknown) Scan parquet spark_catalog.default.date_dim @@ -187,18 +171,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(33) CometFilter +(29) CometFilter Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#7)) -(34) CometProject +(30) CometProject Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(35) ColumnarToRow [codegen id : 1] +(31) ColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(36) BroadcastExchange +(32) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index 5e232e682d..119ae71840 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -4,51 +4,47 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Window [sumsales,i_category] WholeStageCodegen (6) - ColumnarToRow + Sort [i_category,sumsales] InputAdapter - CometSort [i_category,sumsales] - CometColumnarExchange [i_category] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + Exchange [i_category] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index 95e98afea0..d87d02e87b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -1,45 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * HashAggregate (31) - : : +- * ColumnarToRow (30) - : : +- CometColumnarExchange (29) - : : +- RowToColumnar (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (35) - : +- * ColumnarToRow (34) - : +- CometFilter (33) - : +- CometScan parquet spark_catalog.default.customer (32) - +- ReusedExchange (38) +TakeOrderedAndProject (39) ++- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * HashAggregate (29) + : : +- Exchange (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * Project (13) + : : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : : :- * Project (6) + : : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- ReusedExchange (4) + : : : : +- BroadcastExchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store (7) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.household_demographics (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.customer_address (21) + : +- BroadcastExchange (33) + : +- * ColumnarToRow (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.customer (30) + +- ReusedExchange (36) (unknown) Scan parquet spark_catalog.default.store_sales @@ -57,7 +55,7 @@ Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotn (3) ColumnarToRow [codegen id : 5] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -(4) ReusedExchange [Reuses operator id: 46] +(4) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#11] (5) BroadcastHashJoin [codegen id : 5] @@ -169,17 +167,11 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(Un Aggregate Attributes [3]: [sum#19, sum#20, sum#21] Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -(28) RowToColumnar +(28) Exchange Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) CometColumnarExchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) ColumnarToRow [codegen id : 8] -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] - -(31) HashAggregate [codegen id : 8] +(29) HashAggregate [codegen id : 8] Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] @@ -193,52 +185,52 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(33) CometFilter +(31) CometFilter Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) -(34) ColumnarToRow [codegen id : 6] +(32) ColumnarToRow [codegen id : 6] Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -(35) BroadcastExchange +(33) BroadcastExchange Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(35) Project [codegen id : 8] Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35] Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -(38) ReusedExchange [Reuses operator id: 24] +(36) ReusedExchange [Reuses operator id: 24] Output [2]: [ca_address_sk#36, ca_city#37] -(39) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#33] Right keys [1]: [ca_address_sk#36] Join type: Inner Join condition: NOT (ca_city#37 = bought_city#28) -(40) Project [codegen id : 8] +(38) Project [codegen id : 8] Output [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35, ca_address_sk#36, ca_city#37] -(41) TakeOrderedAndProject +(39) TakeOrderedAndProject Input [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] Arguments: 100, [c_last_name#35 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.date_dim (42) +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) (unknown) Scan parquet spark_catalog.default.date_dim @@ -248,18 +240,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter +(41) CometFilter Input [3]: [d_date_sk#11, d_year#38, d_dom#39] Condition : ((((isnotnull(d_dom#39) AND (d_dom#39 >= 1)) AND (d_dom#39 <= 2)) AND d_year#38 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) -(44) CometProject +(42) CometProject Input [3]: [d_date_sk#11, d_year#38, d_dom#39] Arguments: [d_date_sk#11], [d_date_sk#11] -(45) ColumnarToRow [codegen id : 1] +(43) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(46) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt index cad5e40b6e..f2680bebb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -5,57 +5,55 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #2 + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + WholeStageCodegen (5) + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometProject [s_store_sk] + CometFilter [s_city,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index 9b5785dd62..9a32627233 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -1,48 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * ColumnarToRow (42) - +- CometColumnarExchange (41) - +- RowToColumnar (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (25) - : : +- * BroadcastHashJoin LeftAnti BuildRight (24) - : : :- * BroadcastHashJoin LeftAnti BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.customer_address (26) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (25) + : : +- * BroadcastHashJoin LeftAnti BuildRight (24) + : : :- * BroadcastHashJoin LeftAnti BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometProject (28) + : +- CometFilter (27) + : +- CometScan parquet spark_catalog.default.customer_address (26) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_demographics (33) (unknown) Scan parquet spark_catalog.default.customer @@ -69,7 +67,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) ReusedExchange [Reuses operator id: 49] +(6) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#7] (7) BroadcastHashJoin [codegen id : 2] @@ -102,7 +100,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -(13) ReusedExchange [Reuses operator id: 49] +(13) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#11] (14) BroadcastHashJoin [codegen id : 4] @@ -135,7 +133,7 @@ ReadSchema: struct (19) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] -(20) ReusedExchange [Reuses operator id: 49] +(20) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#15] (21) BroadcastHashJoin [codegen id : 6] @@ -229,35 +227,29 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#24] Results [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] -(40) RowToColumnar +(40) Exchange Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] +Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) CometColumnarExchange -Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] -Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) ColumnarToRow [codegen id : 10] -Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] - -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 10] Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] Keys [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#26] Results [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, count(1)#26 AS cnt1#27, cd_purchase_estimate#22, count(1)#26 AS cnt2#28, cd_credit_rating#23, count(1)#26 AS cnt3#29] -(44) TakeOrderedAndProject +(42) TakeOrderedAndProject Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] Arguments: 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (unknown) Scan parquet spark_catalog.default.date_dim @@ -267,18 +259,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(44) CometFilter Input [3]: [d_date_sk#7, d_year#30, d_moy#31] Condition : (((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2001)) AND (d_moy#31 >= 4)) AND (d_moy#31 <= 6)) AND isnotnull(d_date_sk#7)) -(47) CometProject +(45) CometProject Input [3]: [d_date_sk#7, d_year#30, d_moy#31] Arguments: [d_date_sk#7], [d_date_sk#7] -(48) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(49) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index ad64fa5067..f5b4eccfbe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -1,76 +1,74 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] WholeStageCodegen (10) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 + WholeStageCodegen (9) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt index 6d17275156..18ff7c4590 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt @@ -1,36 +1,34 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- * ColumnarToRow (30) - +- CometColumnarExchange (29) - +- RowToColumnar (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan parquet spark_catalog.default.promotion (20) +TakeOrderedAndProject (30) ++- * HashAggregate (29) + +- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * ColumnarToRow (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : +- ReusedExchange (11) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.item (14) + +- BroadcastExchange (24) + +- * ColumnarToRow (23) + +- CometProject (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.promotion (20) (unknown) Scan parquet spark_catalog.default.store_sales @@ -80,7 +78,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(11) ReusedExchange [Reuses operator id: 37] +(11) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#14] (12) BroadcastHashJoin [codegen id : 5] @@ -160,35 +158,29 @@ Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_pr Aggregate Attributes [8]: [sum#20, count#21, sum#22, count#23, sum#24, count#25, sum#26, count#27] Results [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -(28) RowToColumnar +(28) Exchange Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) CometColumnarExchange -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) ColumnarToRow [codegen id : 6] -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] - -(31) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] Keys [1]: [i_item_id#16] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [4]: [avg(ss_quantity#4)#36, avg(UnscaledValue(ss_list_price#5))#37, avg(UnscaledValue(ss_coupon_amt#7))#38, avg(UnscaledValue(ss_sales_price#6))#39] Results [5]: [i_item_id#16, avg(ss_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(ss_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(ss_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(ss_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] -(32) TakeOrderedAndProject +(30) TakeOrderedAndProject Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) (unknown) Scan parquet spark_catalog.default.date_dim @@ -198,18 +190,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(32) CometFilter Input [2]: [d_date_sk#14, d_year#44] Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) -(35) CometProject +(33) CometProject Input [2]: [d_date_sk#14, d_year#44] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(37) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt index eba10dc409..2471de20a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt @@ -1,54 +1,52 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (6) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 422c4bbd10..32499fad8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -1,52 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * ColumnarToRow (45) - +- CometSort (44) - +- CometColumnarExchange (43) - +- RowToColumnar (42) - +- * HashAggregate (41) - +- * ColumnarToRow (40) - +- CometColumnarExchange (39) - +- RowToColumnar (38) - +- * HashAggregate (37) - +- * Expand (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (33) - +- * BroadcastHashJoin LeftSemi BuildRight (32) - :- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (31) - +- * Project (30) - +- * Filter (29) - +- Window (28) - +- * Sort (27) - +- * HashAggregate (26) - +- * ColumnarToRow (25) - +- CometColumnarExchange (24) - +- RowToColumnar (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.store (13) - +- ReusedExchange (19) +TakeOrderedAndProject (42) ++- * Project (41) + +- Window (40) + +- * Sort (39) + +- Exchange (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * Expand (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- ReusedExchange (4) + +- BroadcastExchange (31) + +- * BroadcastHashJoin LeftSemi BuildRight (30) + :- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.store (7) + +- BroadcastExchange (29) + +- * Project (28) + +- * Filter (27) + +- Window (26) + +- * Sort (25) + +- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildRight (20) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store_sales (10) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.store (13) + +- ReusedExchange (19) (unknown) Scan parquet spark_catalog.default.store_sales @@ -64,7 +58,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 8] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 53] +(4) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 8] @@ -134,7 +128,7 @@ Join condition: None Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] -(19) ReusedExchange [Reuses operator id: 53] +(19) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#15] (20) BroadcastHashJoin [codegen id : 4] @@ -154,125 +148,107 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum#16] Results [2]: [s_state#14, sum#17] -(23) RowToColumnar +(23) Exchange Input [2]: [s_state#14, sum#17] +Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(24) CometColumnarExchange -Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(25) ColumnarToRow [codegen id : 5] -Input [2]: [s_state#14, sum#17] - -(26) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 5] Input [2]: [s_state#14, sum#17] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] -(27) Sort [codegen id : 5] +(25) Sort [codegen id : 5] Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 -(28) Window +(26) Window Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] -(29) Filter [codegen id : 6] +(27) Filter [codegen id : 6] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Condition : (ranking#20 <= 5) -(30) Project [codegen id : 6] +(28) Project [codegen id : 6] Output [1]: [s_state#14] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] -(31) BroadcastExchange +(29) BroadcastExchange Input [1]: [s_state#14] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(32) BroadcastHashJoin [codegen id : 7] +(30) BroadcastHashJoin [codegen id : 7] Left keys [1]: [s_state#8] Right keys [1]: [s_state#14] Join type: LeftSemi Join condition: None -(33) BroadcastExchange +(31) BroadcastExchange Input [3]: [s_store_sk#6, s_county#7, s_state#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(35) Project [codegen id : 8] +(33) Project [codegen id : 8] Output [3]: [ss_net_profit#2, s_state#8, s_county#7] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] -(36) Expand [codegen id : 8] +(34) Expand [codegen id : 8] Input [3]: [ss_net_profit#2, s_state#8, s_county#7] Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] -(37) HashAggregate [codegen id : 8] +(35) HashAggregate [codegen id : 8] Input [4]: [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#24] Results [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] -(38) RowToColumnar -Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] - -(39) CometColumnarExchange -Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] -Arguments: hashpartitioning(s_state#21, s_county#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) ColumnarToRow [codegen id : 9] +(36) Exchange Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] +Arguments: hashpartitioning(s_state#21, s_county#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(41) HashAggregate [codegen id : 9] +(37) HashAggregate [codegen id : 9] Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#26] Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS total_sum#27, s_state#21, s_county#22, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS lochierarchy#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS _w0#29, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS _w1#30, CASE WHEN (cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint) = 0) THEN s_state#21 END AS _w2#31] -(42) RowToColumnar -Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] - -(43) CometColumnarExchange -Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] -Arguments: hashpartitioning(_w1#30, _w2#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) CometSort +(38) Exchange Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] -Arguments: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31], [_w1#30 ASC NULLS FIRST, _w2#31 ASC NULLS FIRST, _w0#29 DESC NULLS LAST] +Arguments: hashpartitioning(_w1#30, _w2#31, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(45) ColumnarToRow [codegen id : 10] +(39) Sort [codegen id : 10] Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] +Arguments: [_w1#30 ASC NULLS FIRST, _w2#31 ASC NULLS FIRST, _w0#29 DESC NULLS LAST], false, 0 -(46) Window +(40) Window Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] Arguments: [rank(_w0#29) windowspecdefinition(_w1#30, _w2#31, _w0#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#32], [_w1#30, _w2#31], [_w0#29 DESC NULLS LAST] -(47) Project [codegen id : 11] +(41) Project [codegen id : 11] Output [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] Input [8]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31, rank_within_parent#32] -(48) TakeOrderedAndProject +(42) TakeOrderedAndProject Input [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN s_state#21 END ASC NULLS FIRST, rank_within_parent#32 ASC NULLS FIRST], [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (unknown) Scan parquet spark_catalog.default.date_dim @@ -282,18 +258,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(44) CometFilter Input [2]: [d_date_sk#5, d_month_seq#33] Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1200)) AND (d_month_seq#33 <= 1211)) AND isnotnull(d_date_sk#5)) -(51) CometProject +(45) CometProject Input [2]: [d_date_sk#5, d_month_seq#33] Arguments: [d_date_sk#5], [d_date_sk#5] -(52) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(53) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index 1924ac21cb..0e01c5f710 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -4,77 +4,71 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (10) - ColumnarToRow + Sort [_w1,_w2,_w0] InputAdapter - CometSort [_w1,_w2,_w0] - CometColumnarExchange [_w1,_w2] #1 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + Exchange [_w1,_w2] #1 + WholeStageCodegen (9) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + InputAdapter + Exchange [s_state,s_county,spark_grouping_id] #2 + WholeStageCodegen (8) + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + BroadcastHashJoin [s_state,s_state] + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - BroadcastHashJoin [s_state,s_state] - ColumnarToRow + BroadcastExchange #5 + WholeStageCodegen (6) + Project [s_state] + Filter [ranking] InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #6 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + Window [_w0,s_state] + WholeStageCodegen (5) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + Exchange [s_state] #6 + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 90ae1f6bfa..39bedd1f2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -1,46 +1,42 @@ == Physical Plan == -* ColumnarToRow (42) -+- CometSort (41) - +- CometColumnarExchange (40) - +- RowToColumnar (39) - +- * HashAggregate (38) - +- * ColumnarToRow (37) - +- CometColumnarExchange (36) - +- RowToColumnar (35) - +- * HashAggregate (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildLeft (25) - : :- BroadcastExchange (5) - : : +- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.item (1) - : +- Union (24) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.web_sales (6) - : : +- ReusedExchange (9) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * ColumnarToRow (14) - : : : +- CometFilter (13) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : +- ReusedExchange (15) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * ColumnarToRow (20) - : : +- CometFilter (19) - : : +- CometScan parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (21) - +- BroadcastExchange (31) - +- * ColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.time_dim (27) +* Sort (38) ++- Exchange (37) + +- * HashAggregate (36) + +- Exchange (35) + +- * HashAggregate (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildLeft (25) + : :- BroadcastExchange (5) + : : +- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.item (1) + : +- Union (24) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.web_sales (6) + : : +- ReusedExchange (9) + : :- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * ColumnarToRow (14) + : : : +- CometFilter (13) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : +- ReusedExchange (15) + : +- * Project (23) + : +- * BroadcastHashJoin Inner BuildRight (22) + : :- * ColumnarToRow (20) + : : +- CometFilter (19) + : : +- CometScan parquet spark_catalog.default.store_sales (18) + : +- ReusedExchange (21) + +- BroadcastExchange (31) + +- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.time_dim (27) (unknown) Scan parquet spark_catalog.default.item @@ -80,7 +76,7 @@ Condition : (isnotnull(ws_item_sk#6) AND isnotnull(ws_sold_time_sk#5)) (8) ColumnarToRow [codegen id : 3] Input [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] -(9) ReusedExchange [Reuses operator id: 47] +(9) ReusedExchange [Reuses operator id: 43] Output [1]: [d_date_sk#10] (10) BroadcastHashJoin [codegen id : 3] @@ -108,7 +104,7 @@ Condition : (isnotnull(cs_item_sk#15) AND isnotnull(cs_sold_time_sk#14)) (14) ColumnarToRow [codegen id : 5] Input [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] -(15) ReusedExchange [Reuses operator id: 47] +(15) ReusedExchange [Reuses operator id: 43] Output [1]: [d_date_sk#19] (16) BroadcastHashJoin [codegen id : 5] @@ -136,7 +132,7 @@ Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) (20) ColumnarToRow [codegen id : 7] Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(21) ReusedExchange [Reuses operator id: 47] +(21) ReusedExchange [Reuses operator id: 43] Output [1]: [d_date_sk#28] (22) BroadcastHashJoin [codegen id : 7] @@ -200,45 +196,33 @@ Functions [1]: [partial_sum(UnscaledValue(ext_price#11))] Aggregate Attributes [1]: [sum#36] Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] -(35) RowToColumnar +(35) Exchange Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(36) CometColumnarExchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(37) ColumnarToRow [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] - -(38) HashAggregate [codegen id : 10] +(36) HashAggregate [codegen id : 10] Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] Functions [1]: [sum(UnscaledValue(ext_price#11))] Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#11))#38] Results [5]: [i_brand_id#2 AS brand_id#39, i_brand#3 AS brand#40, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#11))#38,17,2) AS ext_price#41] -(39) RowToColumnar -Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] - -(40) CometColumnarExchange -Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] -Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometSort +(37) Exchange Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] -Arguments: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41], [ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST] +Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(42) ColumnarToRow [codegen id : 11] +(38) Sort [codegen id : 11] Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] +Arguments: [ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) (unknown) Scan parquet spark_catalog.default.date_dim @@ -248,18 +232,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(40) CometFilter Input [3]: [d_date_sk#10, d_year#42, d_moy#43] Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 11)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#10)) -(45) CometProject +(41) CometProject Input [3]: [d_date_sk#10, d_year#42, d_moy#43] Arguments: [d_date_sk#10], [d_date_sk#10] -(46) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(47) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index 797f739ab0..bea5376a00 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -1,73 +1,69 @@ WholeStageCodegen (11) - ColumnarToRow + Sort [ext_price,brand_id] InputAdapter - CometSort [ext_price,brand_id] - CometColumnarExchange [ext_price,brand_id] #1 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] - BroadcastHashJoin [time_sk,t_time_sk] - Project [i_brand_id,i_brand,ext_price,time_sk] - BroadcastHashJoin [i_item_sk,sold_item_sk] + Exchange [ext_price,brand_id] #1 + WholeStageCodegen (10) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] + InputAdapter + Exchange [i_brand,i_brand_id,t_hour,t_minute] #2 + WholeStageCodegen (9) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] + Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] + BroadcastHashJoin [time_sk,t_time_sk] + Project [i_brand_id,i_brand,ext_price,time_sk] + BroadcastHashJoin [i_item_sk,sold_item_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - InputAdapter - Union - WholeStageCodegen (3) - Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_sold_time_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (5) - Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_sold_time_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (7) - Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_sold_time_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + InputAdapter + Union + WholeStageCodegen (3) + Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_sold_time_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (5) + Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + CometFilter [cs_item_sk,cs_sold_time_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (7) + Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_sold_time_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_meal_time,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index dbc785238b..e55a0ca108 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -1,75 +1,74 @@ == Physical Plan == -* ColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometHashAggregate (69) - +- CometColumnarExchange (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometSortMergeJoin (65) - :- CometSort (59) - : +- CometColumnarExchange (58) - : +- RowToColumnar (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * ColumnarToRow (3) - : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * ColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * ColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * ColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * ColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * ColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * ColumnarToRow (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.promotion (51) - +- CometSort (64) - +- CometColumnarExchange (63) - +- CometProject (62) - +- CometFilter (61) - +- CometScan parquet spark_catalog.default.catalog_returns (60) +TakeOrderedAndProject (70) ++- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * Project (66) + +- * SortMergeJoin LeftOuter (65) + :- * Sort (58) + : +- Exchange (57) + : +- * Project (56) + : +- * BroadcastHashJoin LeftOuter BuildRight (55) + : :- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Project (15) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : : :- * Project (9) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : : : : :- * ColumnarToRow (3) + : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- BroadcastExchange (7) + : : : : : : : : : +- * ColumnarToRow (6) + : : : : : : : : : +- CometFilter (5) + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) + : : : : : : : : +- BroadcastExchange (13) + : : : : : : : : +- * ColumnarToRow (12) + : : : : : : : : +- CometFilter (11) + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * ColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- CometProject (24) + : : : : : : +- CometFilter (23) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) + : : : : : +- BroadcastExchange (33) + : : : : : +- * ColumnarToRow (32) + : : : : : +- CometProject (31) + : : : : : +- CometFilter (30) + : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) + : : : : +- ReusedExchange (36) + : : : +- BroadcastExchange (42) + : : : +- * ColumnarToRow (41) + : : : +- CometFilter (40) + : : : +- CometScan parquet spark_catalog.default.date_dim (39) + : : +- BroadcastExchange (48) + : : +- * ColumnarToRow (47) + : : +- CometFilter (46) + : : +- CometScan parquet spark_catalog.default.date_dim (45) + : +- BroadcastExchange (54) + : +- * ColumnarToRow (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.promotion (51) + +- * Sort (64) + +- Exchange (63) + +- * ColumnarToRow (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.catalog_returns (59) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -236,7 +235,7 @@ Join condition: None Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -(36) ReusedExchange [Reuses operator id: 76] +(36) ReusedExchange [Reuses operator id: 75] Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (37) BroadcastHashJoin [codegen id : 10] @@ -333,16 +332,13 @@ Join condition: None Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] -(57) RowToColumnar +(57) Exchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(58) CometColumnarExchange +(58) Sort [codegen id : 11] Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(59) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] +Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] @@ -351,81 +347,86 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(61) CometFilter +(60) CometFilter Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) -(62) CometProject +(61) CometProject Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] -(63) CometColumnarExchange +(62) ColumnarToRow [codegen id : 12] Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(64) CometSort +(63) Exchange Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(65) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter +(64) Sort [codegen id : 13] +Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST], false, 0 -(66) CometProject +(65) SortMergeJoin [codegen id : 14] +Left keys [2]: [cs_item_sk#4, cs_order_number#6] +Right keys [2]: [cr_item_sk#30, cr_order_number#31] +Join type: LeftOuter +Join condition: None + +(66) Project [codegen id : 14] +Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(67) CometHashAggregate +(67) HashAggregate [codegen id : 14] Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#33] +Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -(68) CometColumnarExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(68) Exchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(69) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] +(69) HashAggregate [codegen id : 15] +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#35] +Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#35 AS no_promo#36, count(1)#35 AS promo#37, count(1)#35 AS total_cnt#38] -(70) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), 100, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -(71) ColumnarToRow [codegen id : 11] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] +(70) TakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] +Arguments: 100, [total_cnt#38 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(73) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(72) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +Condition : ((((isnotnull(d_year#39) AND (d_year#39 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(74) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +(73) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(75) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(76) BroadcastExchange +(75) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index 5bf7ab343a..17fc9dee7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -1,108 +1,116 @@ -WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometColumnarExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - RowToColumnar - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - ColumnarToRow - InputAdapter - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] +TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] + WholeStageCodegen (15) + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] + InputAdapter + Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + WholeStageCodegen (14) + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] + Project [w_warehouse_name,i_item_desc,d_week_seq] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (11) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #2 + WholeStageCodegen (10) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + ColumnarToRow + InputAdapter + CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + BroadcastExchange #6 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) + BroadcastExchange #7 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometProject [cd_demo_sk] + CometFilter [cd_marital_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) + BroadcastExchange #8 + WholeStageCodegen (5) ColumnarToRow InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + BroadcastExchange #9 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [d_week_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) + BroadcastExchange #10 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [d_week_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) + BroadcastExchange #11 + WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometColumnarExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + InputAdapter + WholeStageCodegen (13) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #12 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index ffdecea92e..dc64c33861 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt @@ -1,40 +1,36 @@ == Physical Plan == -* ColumnarToRow (36) -+- CometSort (35) - +- CometColumnarExchange (34) - +- RowToColumnar (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (26) - : +- * HashAggregate (25) - : +- * ColumnarToRow (24) - : +- CometColumnarExchange (23) - : +- RowToColumnar (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * ColumnarToRow (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.customer (27) +* Sort (32) ++- Exchange (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * Filter (24) + : +- * HashAggregate (23) + : +- Exchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (28) + +- * ColumnarToRow (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (unknown) Scan parquet spark_catalog.default.store_sales @@ -52,7 +48,7 @@ Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnu (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 41] +(4) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -136,24 +132,18 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#14] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) RowToColumnar +(22) Exchange Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(23) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) ColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(25) HashAggregate [codegen id : 6] +(23) HashAggregate [codegen id : 6] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(26) Filter [codegen id : 6] +(24) Filter [codegen id : 6] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) @@ -164,49 +154,43 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(28) CometFilter +(26) CometFilter Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Condition : isnotnull(c_customer_sk#18) -(29) ColumnarToRow [codegen id : 5] +(27) ColumnarToRow [codegen id : 5] Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(30) BroadcastExchange +(28) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(30) Project [codegen id : 6] Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(33) RowToColumnar -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] - -(34) CometColumnarExchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(35) CometSort +(31) Exchange Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(36) ColumnarToRow [codegen id : 7] +(32) Sort [codegen id : 7] Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: [cnt#17 DESC NULLS LAST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (41) -+- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (unknown) Scan parquet spark_catalog.default.date_dim @@ -216,18 +200,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(38) CometFilter +(34) CometFilter Input [3]: [d_date_sk#7, d_year#23, d_dom#24] Condition : ((((isnotnull(d_dom#24) AND (d_dom#24 >= 1)) AND (d_dom#24 <= 2)) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(39) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#23, d_dom#24] Arguments: [d_date_sk#7], [d_date_sk#7] -(40) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(41) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index da73b27ba1..7c5ee1ef5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -1,60 +1,56 @@ WholeStageCodegen (7) - ColumnarToRow + Sort [cnt] InputAdapter - CometSort [cnt] - CometColumnarExchange [cnt] #1 - RowToColumnar - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 + Exchange [cnt] #1 + WholeStageCodegen (6) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index 54db51cb33..85413ac1ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -1,83 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * ColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- RowToColumnar (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * ColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- RowToColumnar (32) - : : +- * HashAggregate (31) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Project (27) - : : : +- * BroadcastHashJoin Inner BuildRight (26) - : : : :- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.store_sales (22) - : : +- ReusedExchange (28) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * ColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- RowToColumnar (51) - : +- * HashAggregate (50) - : +- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (46) - : : +- * BroadcastHashJoin Inner BuildRight (45) - : : :- * ColumnarToRow (40) - : : : +- CometFilter (39) - : : : +- CometScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (44) - : : +- * ColumnarToRow (43) - : : +- CometFilter (42) - : : +- CometScan parquet spark_catalog.default.web_sales (41) - : +- ReusedExchange (47) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * ColumnarToRow (74) - +- CometColumnarExchange (73) - +- RowToColumnar (72) - +- * HashAggregate (71) - +- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * ColumnarToRow (61) - : : +- CometFilter (60) - : : +- CometScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (65) - : +- * ColumnarToRow (64) - : +- CometFilter (63) - : +- CometScan parquet spark_catalog.default.web_sales (62) - +- ReusedExchange (68) +TakeOrderedAndProject (71) ++- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (25) + : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- BroadcastExchange (23) + : : : +- * ColumnarToRow (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- ReusedExchange (26) + : +- BroadcastExchange (50) + : +- * Filter (49) + : +- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * ColumnarToRow (36) + : : : +- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (43) + +- BroadcastExchange (68) + +- * HashAggregate (67) + +- Exchange (66) + +- * HashAggregate (65) + +- * Project (64) + +- * BroadcastHashJoin Inner BuildRight (63) + :- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * ColumnarToRow (55) + : : +- CometFilter (54) + : : +- CometScan parquet spark_catalog.default.customer (53) + : +- BroadcastExchange (59) + : +- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (62) (unknown) Scan parquet spark_catalog.default.customer @@ -123,7 +115,7 @@ Join condition: None Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 83] +(10) ReusedExchange [Reuses operator id: 75] Output [2]: [d_date_sk#9, d_year#10] (11) BroadcastHashJoin [codegen id : 3] @@ -143,24 +135,18 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] Aggregate Attributes [1]: [sum#11] Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -(14) RowToColumnar +(14) Exchange Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 16] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] - -(17) HashAggregate [codegen id : 16] +(15) HashAggregate [codegen id : 16] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] -(18) Filter [codegen id : 16] +(16) Filter [codegen id : 16] Input [2]: [customer_id#14, year_total#15] Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) @@ -171,11 +157,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(20) CometFilter +(18) CometFilter Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) -(21) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 6] Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] (unknown) Scan parquet spark_catalog.default.store_sales @@ -186,69 +172,63 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter +(21) CometFilter Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) -(24) ColumnarToRow [codegen id : 4] +(22) ColumnarToRow [codegen id : 4] Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -(25) BroadcastExchange +(23) BroadcastExchange Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [c_customer_sk#16] Right keys [1]: [ss_customer_sk#20] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(25) Project [codegen id : 6] Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -(28) ReusedExchange [Reuses operator id: 87] +(26) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#24, d_year#25] -(29) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#22] Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None -(30) Project [codegen id : 6] +(28) Project [codegen id : 6] Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] -(31) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] Aggregate Attributes [1]: [sum#26] Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -(32) RowToColumnar -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] - -(33) CometColumnarExchange -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) ColumnarToRow [codegen id : 7] +(30) Exchange Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] +Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] -(36) BroadcastExchange +(32) BroadcastExchange Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(33) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#28] Join type: Inner @@ -261,11 +241,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(39) CometFilter +(35) CometFilter Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) -(40) ColumnarToRow [codegen id : 10] +(36) ColumnarToRow [codegen id : 10] Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] (unknown) Scan parquet spark_catalog.default.web_sales @@ -276,79 +256,73 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(42) CometFilter +(38) CometFilter Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Condition : isnotnull(ws_bill_customer_sk#36) -(43) ColumnarToRow [codegen id : 8] +(39) ColumnarToRow [codegen id : 8] Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -(44) BroadcastExchange +(40) BroadcastExchange Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 10] +(41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#32] Right keys [1]: [ws_bill_customer_sk#36] Join type: Inner Join condition: None -(46) Project [codegen id : 10] +(42) Project [codegen id : 10] Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -(47) ReusedExchange [Reuses operator id: 83] +(43) ReusedExchange [Reuses operator id: 75] Output [2]: [d_date_sk#40, d_year#41] -(48) BroadcastHashJoin [codegen id : 10] +(44) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#38] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(49) Project [codegen id : 10] +(45) Project [codegen id : 10] Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] -(50) HashAggregate [codegen id : 10] +(46) HashAggregate [codegen id : 10] Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] Aggregate Attributes [1]: [sum#42] Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -(51) RowToColumnar +(47) Exchange Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(52) CometColumnarExchange -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) ColumnarToRow [codegen id : 11] -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] - -(54) HashAggregate [codegen id : 11] +(48) HashAggregate [codegen id : 11] Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] -(55) Filter [codegen id : 11] +(49) Filter [codegen id : 11] Input [2]: [customer_id#45, year_total#46] Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) -(56) BroadcastExchange +(50) BroadcastExchange Input [2]: [customer_id#45, year_total#46] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#45] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(52) Project [codegen id : 16] Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] @@ -359,11 +333,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(60) CometFilter +(54) CometFilter Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) -(61) ColumnarToRow [codegen id : 14] +(55) ColumnarToRow [codegen id : 14] Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] (unknown) Scan parquet spark_catalog.default.web_sales @@ -374,89 +348,83 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(63) CometFilter +(57) CometFilter Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Condition : isnotnull(ws_bill_customer_sk#51) -(64) ColumnarToRow [codegen id : 12] +(58) ColumnarToRow [codegen id : 12] Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -(65) BroadcastExchange +(59) BroadcastExchange Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(66) BroadcastHashJoin [codegen id : 14] +(60) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_customer_sk#47] Right keys [1]: [ws_bill_customer_sk#51] Join type: Inner Join condition: None -(67) Project [codegen id : 14] +(61) Project [codegen id : 14] Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -(68) ReusedExchange [Reuses operator id: 87] +(62) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#55, d_year#56] -(69) BroadcastHashJoin [codegen id : 14] +(63) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_sold_date_sk#53] Right keys [1]: [d_date_sk#55] Join type: Inner Join condition: None -(70) Project [codegen id : 14] +(64) Project [codegen id : 14] Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] -(71) HashAggregate [codegen id : 14] +(65) HashAggregate [codegen id : 14] Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] Aggregate Attributes [1]: [sum#57] Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -(72) RowToColumnar -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] - -(73) CometColumnarExchange -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) ColumnarToRow [codegen id : 15] +(66) Exchange Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(75) HashAggregate [codegen id : 15] +(67) HashAggregate [codegen id : 15] Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#52))#44] Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#52))#44,17,2) AS year_total#60] -(76) BroadcastExchange +(68) BroadcastExchange Input [2]: [customer_id#59, year_total#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(69) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#59] Join type: Inner Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) -(78) Project [codegen id : 16] +(70) Project [codegen id : 16] Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] -(79) TakeOrderedAndProject +(71) TakeOrderedAndProject Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (83) -+- * ColumnarToRow (82) - +- CometFilter (81) - +- CometScan parquet spark_catalog.default.date_dim (80) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim @@ -466,22 +434,22 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter +(73) CometFilter Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(82) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(83) BroadcastExchange +(75) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (87) -+- * ColumnarToRow (86) - +- CometFilter (85) - +- CometScan parquet spark_catalog.default.date_dim (84) +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) (unknown) Scan parquet spark_catalog.default.date_dim @@ -491,19 +459,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(85) CometFilter +(77) CometFilter Input [2]: [d_date_sk#24, d_year#25] Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) -(86) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] -(87) BroadcastExchange +(79) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 1877fb7e14..9d3ae8fbee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -7,11 +7,41 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - RowToColumnar - WholeStageCodegen (3) + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 + WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -22,109 +52,71 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #6 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 WholeStageCodegen (11) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - RowToColumnar - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index 8b2d451b09..0ff759f363 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -1,138 +1,133 @@ == Physical Plan == -TakeOrderedAndProject (134) -+- * Project (133) - +- * SortMergeJoin Inner (132) - :- * ColumnarToRow (72) - : +- CometSort (71) - : +- CometColumnarExchange (70) - : +- CometFilter (69) - : +- CometHashAggregate (68) - : +- CometColumnarExchange (67) - : +- CometHashAggregate (66) - : +- CometHashAggregate (65) - : +- CometColumnarExchange (64) - : +- CometHashAggregate (63) - : +- CometUnion (62) - : :- CometProject (23) - : : +- CometSortMergeJoin (22) - : : :- CometSort (16) - : : : +- CometColumnarExchange (15) - : : : +- RowToColumnar (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- CometSort (21) - : : +- CometColumnarExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan parquet spark_catalog.default.catalog_returns (17) - : :- CometProject (42) - : : +- CometSortMergeJoin (41) - : : :- CometSort (35) - : : : +- CometColumnarExchange (34) - : : : +- RowToColumnar (33) - : : : +- * Project (32) - : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : :- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * ColumnarToRow (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) - : : : : +- ReusedExchange (27) - : : : +- ReusedExchange (30) - : : +- CometSort (40) - : : +- CometColumnarExchange (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.store_returns (36) - : +- CometProject (61) - : +- CometSortMergeJoin (60) - : :- CometSort (54) - : : +- CometColumnarExchange (53) - : : +- RowToColumnar (52) - : : +- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * ColumnarToRow (45) - : : : : +- CometFilter (44) - : : : : +- CometScan parquet spark_catalog.default.web_sales (43) - : : : +- ReusedExchange (46) - : : +- ReusedExchange (49) - : +- CometSort (59) - : +- CometColumnarExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan parquet spark_catalog.default.web_returns (55) - +- * ColumnarToRow (131) - +- CometSort (130) - +- CometColumnarExchange (129) - +- CometFilter (128) - +- CometHashAggregate (127) - +- CometColumnarExchange (126) - +- CometHashAggregate (125) - +- CometHashAggregate (124) - +- CometColumnarExchange (123) - +- CometHashAggregate (122) - +- CometUnion (121) - :- CometProject (88) - : +- CometSortMergeJoin (87) - : :- CometSort (84) - : : +- CometColumnarExchange (83) - : : +- RowToColumnar (82) - : : +- * Project (81) - : : +- * BroadcastHashJoin Inner BuildRight (80) - : : :- * Project (78) - : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : :- * ColumnarToRow (75) - : : : : +- CometFilter (74) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) - : : : +- ReusedExchange (76) - : : +- ReusedExchange (79) - : +- CometSort (86) - : +- ReusedExchange (85) - :- CometProject (104) - : +- CometSortMergeJoin (103) - : :- CometSort (100) - : : +- CometColumnarExchange (99) - : : +- RowToColumnar (98) - : : +- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * Project (94) - : : : +- * BroadcastHashJoin Inner BuildRight (93) - : : : :- * ColumnarToRow (91) - : : : : +- CometFilter (90) - : : : : +- CometScan parquet spark_catalog.default.store_sales (89) - : : : +- ReusedExchange (92) - : : +- ReusedExchange (95) - : +- CometSort (102) - : +- ReusedExchange (101) - +- CometProject (120) - +- CometSortMergeJoin (119) - :- CometSort (116) - : +- CometColumnarExchange (115) - : +- RowToColumnar (114) - : +- * Project (113) - : +- * BroadcastHashJoin Inner BuildRight (112) - : :- * Project (110) - : : +- * BroadcastHashJoin Inner BuildRight (109) - : : :- * ColumnarToRow (107) - : : : +- CometFilter (106) - : : : +- CometScan parquet spark_catalog.default.web_sales (105) - : : +- ReusedExchange (108) - : +- ReusedExchange (111) - +- CometSort (118) - +- ReusedExchange (117) +TakeOrderedAndProject (129) ++- * Project (128) + +- * SortMergeJoin Inner (127) + :- * Sort (71) + : +- Exchange (70) + : +- * Filter (69) + : +- * HashAggregate (68) + : +- Exchange (67) + : +- * HashAggregate (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- Union (62) + : :- * Project (23) + : : +- * SortMergeJoin LeftOuter (22) + : : :- * Sort (15) + : : : +- Exchange (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- ReusedExchange (11) + : : +- * Sort (21) + : : +- Exchange (20) + : : +- * ColumnarToRow (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- * Project (42) + : : +- * SortMergeJoin LeftOuter (41) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : :- * Project (29) + : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : :- * ColumnarToRow (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- * Sort (40) + : : +- Exchange (39) + : : +- * ColumnarToRow (38) + : : +- CometProject (37) + : : +- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.store_returns (35) + : +- * Project (61) + : +- * SortMergeJoin LeftOuter (60) + : :- * Sort (53) + : : +- Exchange (52) + : : +- * Project (51) + : : +- * BroadcastHashJoin Inner BuildRight (50) + : : :- * Project (48) + : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : :- * ColumnarToRow (45) + : : : : +- CometFilter (44) + : : : : +- CometScan parquet spark_catalog.default.web_sales (43) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- * Sort (59) + : +- Exchange (58) + : +- * ColumnarToRow (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.web_returns (54) + +- * Sort (126) + +- Exchange (125) + +- * Filter (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- * HashAggregate (120) + +- Exchange (119) + +- * HashAggregate (118) + +- Union (117) + :- * Project (86) + : +- * SortMergeJoin LeftOuter (85) + : :- * Sort (82) + : : +- Exchange (81) + : : +- * Project (80) + : : +- * BroadcastHashJoin Inner BuildRight (79) + : : :- * Project (77) + : : : +- * BroadcastHashJoin Inner BuildRight (76) + : : : :- * ColumnarToRow (74) + : : : : +- CometFilter (73) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) + : : : +- ReusedExchange (75) + : : +- ReusedExchange (78) + : +- * Sort (84) + : +- ReusedExchange (83) + :- * Project (101) + : +- * SortMergeJoin LeftOuter (100) + : :- * Sort (97) + : : +- Exchange (96) + : : +- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * Project (92) + : : : +- * BroadcastHashJoin Inner BuildRight (91) + : : : :- * ColumnarToRow (89) + : : : : +- CometFilter (88) + : : : : +- CometScan parquet spark_catalog.default.store_sales (87) + : : : +- ReusedExchange (90) + : : +- ReusedExchange (93) + : +- * Sort (99) + : +- ReusedExchange (98) + +- * Project (116) + +- * SortMergeJoin LeftOuter (115) + :- * Sort (112) + : +- Exchange (111) + : +- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * Project (107) + : : +- * BroadcastHashJoin Inner BuildRight (106) + : : :- * ColumnarToRow (104) + : : : +- CometFilter (103) + : : : +- CometScan parquet spark_catalog.default.web_sales (102) + : : +- ReusedExchange (105) + : +- ReusedExchange (108) + +- * Sort (114) + +- ReusedExchange (113) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -182,7 +177,7 @@ Join condition: None Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 138] +(11) ReusedExchange [Reuses operator id: 133] Output [2]: [d_date_sk#13, d_year#14] (12) BroadcastHashJoin [codegen id : 3] @@ -195,16 +190,13 @@ Join condition: None Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -(14) RowToColumnar +(14) Exchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange +(15) Sort [codegen id : 4] Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] @@ -213,30 +205,34 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(18) CometFilter +(17) CometFilter Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) -(19) CometProject +(18) CometProject Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(20) CometColumnarExchange +(19) ColumnarToRow [codegen id : 5] +Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] + +(20) Exchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometSort +(21) Sort [codegen id : 6] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] +Arguments: [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST], false, 0 -(22) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter +(22) SortMergeJoin [codegen id : 7] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#16, cr_item_sk#15] +Join type: LeftOuter +Join condition: None -(23) CometProject +(23) Project [codegen id : 7] +Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] (unknown) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] @@ -250,45 +246,42 @@ ReadSchema: struct -(37) CometFilter +(36) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(38) CometProject +(37) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(39) CometColumnarExchange +(38) ColumnarToRow [codegen id : 12] +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] + +(39) Exchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(40) CometSort +(40) Sort [codegen id : 13] Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] +Arguments: [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST], false, 0 -(41) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter +(41) SortMergeJoin [codegen id : 14] +Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] +Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] +Join type: LeftOuter +Join condition: None -(42) CometProject +(42) Project [codegen id : 14] +Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] (unknown) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] @@ -334,45 +331,42 @@ ReadSchema: struct -(56) CometFilter +(55) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(57) CometProject +(56) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(58) CometColumnarExchange +(57) ColumnarToRow [codegen id : 19] Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(59) CometSort +(58) Exchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(60) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter +(59) Sort [codegen id : 20] +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST], false, 0 + +(60) SortMergeJoin [codegen id : 21] +Left keys [2]: [ws_order_number#43, ws_item_sk#42] +Right keys [2]: [wr_order_number#56, wr_item_sk#55] +Join type: LeftOuter +Join condition: None -(61) CometProject +(61) Project [codegen id : 21] +Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] -(62) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] +(62) Union -(63) CometHashAggregate +(63) HashAggregate [codegen id : 22] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(64) CometColumnarExchange +(64) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(65) CometHashAggregate +(65) HashAggregate [codegen id : 23] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(66) CometHashAggregate +(66) HashAggregate [codegen id : 23] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum#62, sum#63] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -(67) CometColumnarExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(67) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(68) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] +(68) HashAggregate [codegen id : 24] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] -(69) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) +(69) Filter [codegen id : 24] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Condition : isnotnull(sales_cnt#68) -(70) CometColumnarExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(70) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(71) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(72) ColumnarToRow [codegen id : 10] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +(71) Sort [codegen id : 25] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(74) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) +(73) CometFilter +Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Condition : isnotnull(cs_item_sk#70) -(75) ColumnarToRow [codegen id : 13] -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +(74) ColumnarToRow [codegen id : 28] +Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] -(76) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +(75) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(77) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_item_sk#66] -Right keys [1]: [i_item_sk#72] +(76) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [cs_item_sk#70] +Right keys [1]: [i_item_sk#76] Join type: Inner Join condition: None -(78) Project [codegen id : 13] -Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +(77) Project [codegen id : 28] +Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(79) ReusedExchange [Reuses operator id: 142] -Output [2]: [d_date_sk#77, d_year#78] +(78) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#81, d_year#82] -(80) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_sold_date_sk#70] -Right keys [1]: [d_date_sk#77] +(79) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [cs_sold_date_sk#74] +Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(81) Project [codegen id : 13] -Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] - -(82) RowToColumnar -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +(80) Project [codegen id : 28] +Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] -(83) CometColumnarExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(81) Exchange +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(84) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] +(82) Sort [codegen id : 29] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 -(85) ReusedExchange [Reuses operator id: 20] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +(83) ReusedExchange [Reuses operator id: 20] +Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(86) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] +(84) Sort [codegen id : 31] +Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +Arguments: [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST], false, 0 -(87) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter +(85) SortMergeJoin [codegen id : 32] +Left keys [2]: [cs_order_number#71, cs_item_sk#70] +Right keys [2]: [cr_order_number#84, cr_item_sk#83] +Join type: LeftOuter +Join condition: None -(88) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] +(86) Project [codegen id : 32] +Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#20, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#21] +Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] (unknown) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] +PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(90) CometFilter -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) +(88) CometFilter +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Condition : isnotnull(ss_item_sk#87) -(91) ColumnarToRow [codegen id : 16] -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +(89) ColumnarToRow [codegen id : 35] +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -(92) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +(90) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(93) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ss_item_sk#83] -Right keys [1]: [i_item_sk#89] +(91) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ss_item_sk#87] +Right keys [1]: [i_item_sk#93] Join type: Inner Join condition: None -(94) Project [codegen id : 16] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +(92) Project [codegen id : 35] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(95) ReusedExchange [Reuses operator id: 142] -Output [2]: [d_date_sk#94, d_year#95] +(93) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#98, d_year#99] -(96) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ss_sold_date_sk#87] -Right keys [1]: [d_date_sk#94] +(94) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ss_sold_date_sk#91] +Right keys [1]: [d_date_sk#98] Join type: Inner Join condition: None -(97) Project [codegen id : 16] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] +(95) Project [codegen id : 35] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] -(98) RowToColumnar -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +(96) Exchange +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(99) CometColumnarExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +(97) Sort [codegen id : 36] +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 -(100) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] +(98) ReusedExchange [Reuses operator id: 39] +Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -(101) ReusedExchange [Reuses operator id: 39] -Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +(99) Sort [codegen id : 38] +Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +Arguments: [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST], false, 0 -(102) CometSort -Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] - -(103) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter +(100) SortMergeJoin [codegen id : 39] +Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] +Right keys [2]: [sr_ticket_number#101, sr_item_sk#100] +Join type: LeftOuter +Join condition: None -(104) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] +(101) Project [codegen id : 39] +Output [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#40, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#41] +Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] (unknown) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] +PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(106) CometFilter -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Condition : isnotnull(ws_item_sk#100) +(103) CometFilter +Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Condition : isnotnull(ws_item_sk#104) -(107) ColumnarToRow [codegen id : 19] -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +(104) ColumnarToRow [codegen id : 42] +Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -(108) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +(105) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(109) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_item_sk#100] -Right keys [1]: [i_item_sk#106] +(106) BroadcastHashJoin [codegen id : 42] +Left keys [1]: [ws_item_sk#104] +Right keys [1]: [i_item_sk#110] Join type: Inner Join condition: None -(110) Project [codegen id : 19] -Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +(107) Project [codegen id : 42] +Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(111) ReusedExchange [Reuses operator id: 142] -Output [2]: [d_date_sk#111, d_year#112] +(108) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#115, d_year#116] -(112) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#104] -Right keys [1]: [d_date_sk#111] +(109) BroadcastHashJoin [codegen id : 42] +Left keys [1]: [ws_sold_date_sk#108] +Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None -(113) Project [codegen id : 19] -Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] - -(114) RowToColumnar -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +(110) Project [codegen id : 42] +Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] -(115) CometColumnarExchange -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +(111) Exchange +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(116) CometSort -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] +(112) Sort [codegen id : 43] +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST], false, 0 -(117) ReusedExchange [Reuses operator id: 58] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +(113) ReusedExchange [Reuses operator id: 58] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(118) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] +(114) Sort [codegen id : 45] +Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST], false, 0 -(119) CometSortMergeJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter +(115) SortMergeJoin [codegen id : 46] +Left keys [2]: [ws_order_number#105, ws_item_sk#104] +Right keys [2]: [wr_order_number#118, wr_item_sk#117] +Join type: LeftOuter +Join condition: None -(120) CometProject -Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] +(116) Project [codegen id : 46] +Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#60, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#61] +Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(121) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] +(117) Union -(122) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +(118) HashAggregate [codegen id : 47] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(123) CometColumnarExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(119) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(124) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +(120) HashAggregate [codegen id : 48] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(125) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +(121) HashAggregate [codegen id : 48] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum#62, sum#121] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -(126) CometColumnarExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(122) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(127) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +(123) HashAggregate [codegen id : 49] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] +Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#20)#66 AS sales_cnt#123, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#124] -(128) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Condition : isnotnull(sales_cnt#118) - -(129) CometColumnarExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] +(124) Filter [codegen id : 49] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] +Condition : isnotnull(sales_cnt#123) -(130) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] +(125) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] +Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(131) ColumnarToRow [codegen id : 20] -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +(126) Sort [codegen id : 50] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] +Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 -(132) SortMergeJoin [codegen id : 21] +(127) SortMergeJoin [codegen id : 51] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right keys [4]: [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Join type: Inner -Join condition: ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) -(133) Project [codegen id : 21] -Output [10]: [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +(128) Project [codegen id : 51] +Output [10]: [d_year#82 AS prev_year#125, d_year#14 AS year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#123 AS prev_yr_cnt#127, sales_cnt#68 AS curr_yr_cnt#128, (sales_cnt#68 - sales_cnt#123) AS sales_cnt_diff#129, (sales_amt#69 - sales_amt#124) AS sales_amt_diff#130] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -(134) TakeOrderedAndProject -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -Arguments: 100, [sales_cnt_diff#124 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] +(129) TakeOrderedAndProject +Input [10]: [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] +Arguments: 100, [sales_cnt_diff#129 ASC NULLS FIRST], [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (138) -+- * ColumnarToRow (137) - +- CometFilter (136) - +- CometScan parquet spark_catalog.default.date_dim (135) +BroadcastExchange (133) ++- * ColumnarToRow (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) (unknown) Scan parquet spark_catalog.default.date_dim @@ -736,14 +732,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(136) CometFilter +(131) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(137) ColumnarToRow [codegen id : 1] +(132) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(138) BroadcastExchange +(133) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] @@ -751,33 +747,33 @@ Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (142) -+- * ColumnarToRow (141) - +- CometFilter (140) - +- CometScan parquet spark_catalog.default.date_dim (139) +Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (137) ++- * ColumnarToRow (136) + +- CometFilter (135) + +- CometScan parquet spark_catalog.default.date_dim (134) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] +Output [2]: [d_date_sk#81, d_year#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(140) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) +(135) CometFilter +Input [2]: [d_date_sk#81, d_year#82] +Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(141) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] +(136) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#81, d_year#82] -(142) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] +(137) BroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 89 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 +Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 +Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt index 153383e5d0..44bcabcdb1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt @@ -1,185 +1,240 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_amt_diff] - WholeStageCodegen (21) + WholeStageCodegen (51) Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter - WholeStageCodegen (10) - ColumnarToRow + WholeStageCodegen (25) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - CometSort [i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [sales_cnt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - RowToColumnar - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + WholeStageCodegen (24) + Filter [sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + WholeStageCodegen (23) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + WholeStageCodegen (22) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (7) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (3) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + WholeStageCodegen (6) + Sort [cr_order_number,cr_item_sk] InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow + Exchange [cr_order_number,cr_item_sk] #7 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + WholeStageCodegen (14) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (11) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (10) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_order_number,cr_item_sk] - CometColumnarExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - RowToColumnar - WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow + WholeStageCodegen (13) + Sort [sr_ticket_number,sr_item_sk] InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Exchange [sr_ticket_number,sr_item_sk] #9 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + WholeStageCodegen (21) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - RowToColumnar - WholeStageCodegen (9) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow + WholeStageCodegen (18) + Sort [ws_order_number,ws_item_sk] InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Exchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (17) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_order_number,wr_item_sk] - CometColumnarExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + WholeStageCodegen (20) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #11 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter - WholeStageCodegen (20) - ColumnarToRow + WholeStageCodegen (50) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - CometSort [i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [sales_cnt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - RowToColumnar - WholeStageCodegen (13) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + WholeStageCodegen (49) + Filter [sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + WholeStageCodegen (48) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 + WholeStageCodegen (47) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (32) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (29) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (28) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + InputAdapter + WholeStageCodegen (31) + Sort [cr_order_number,cr_item_sk] InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - ColumnarToRow + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + WholeStageCodegen (39) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (36) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (35) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_order_number,cr_item_sk] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - RowToColumnar - WholeStageCodegen (16) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow + WholeStageCodegen (38) + Sort [sr_ticket_number,sr_item_sk] InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + WholeStageCodegen (46) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - RowToColumnar - WholeStageCodegen (19) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow + WholeStageCodegen (43) + Sort [ws_order_number,ws_item_sk] InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Exchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (42) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_order_number,wr_item_sk] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + WholeStageCodegen (45) + Sort [wr_order_number,wr_item_sk] + InputAdapter + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index 55745424ea..459bc5c019 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -1,44 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- * ColumnarToRow (38) - +- CometColumnarExchange (37) - +- RowToColumnar (36) - +- * HashAggregate (35) - +- Union (34) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (13) - : +- * ColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.date_dim (10) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.web_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * ColumnarToRow (27) - : : +- CometFilter (26) - : : +- CometScan parquet spark_catalog.default.catalog_sales (25) - : +- ReusedExchange (28) - +- ReusedExchange (31) +TakeOrderedAndProject (38) ++- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- Union (34) + :- * Project (15) + : +- * BroadcastHashJoin Inner BuildRight (14) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.item (4) + : +- BroadcastExchange (13) + : +- * ColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.date_dim (10) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.web_sales (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * ColumnarToRow (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.catalog_sales (25) + : +- ReusedExchange (28) + +- ReusedExchange (31) (unknown) Scan parquet spark_catalog.default.store_sales @@ -203,24 +201,18 @@ Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#12)) Aggregate Attributes [2]: [count#37, sum#38] Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] -(36) RowToColumnar +(36) Exchange Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] +Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(37) CometColumnarExchange -Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] -Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(38) ColumnarToRow [codegen id : 11] -Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] - -(39) HashAggregate [codegen id : 11] +(37) HashAggregate [codegen id : 11] Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] Keys [5]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] Aggregate Attributes [2]: [count(1)#41, sum(UnscaledValue(ext_sales_price#12))#42] Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count(1)#41 AS sales_cnt#43, MakeDecimal(sum(UnscaledValue(ext_sales_price#12))#42,17,2) AS sales_amt#44] -(40) TakeOrderedAndProject +(38) TakeOrderedAndProject Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] Arguments: 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index 0b6532be90..73e6b09afe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -1,60 +1,58 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] WholeStageCodegen (11) HashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] [count(1),sum(UnscaledValue(ext_sales_price)),sales_cnt,sales_amt,count,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,col_name,d_year,d_qoy,i_category] #1 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] - InputAdapter - Union - WholeStageCodegen (3) - Project [ss_store_sk,d_year,d_qoy,i_category,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + InputAdapter + Exchange [channel,col_name,d_year,d_qoy,i_category] #1 + WholeStageCodegen (10) + HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] + InputAdapter + Union + WholeStageCodegen (3) + Project [ss_store_sk,d_year,d_qoy,i_category,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - WholeStageCodegen (6) - Project [ws_ship_customer_sk,d_year,d_qoy,i_category,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_ship_customer_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_category] #2 + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + WholeStageCodegen (6) + Project [ws_ship_customer_sk,d_year,d_qoy,i_category,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_ship_customer_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - WholeStageCodegen (9) - Project [cs_ship_addr_sk,d_year,d_qoy,i_category,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_ship_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [i_item_sk,i_category] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + WholeStageCodegen (9) + Project [cs_ship_addr_sk,d_year,d_qoy,i_category,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_ship_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 + ReusedExchange [i_item_sk,i_category] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index 66cfac4c6b..692db69b81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -1,103 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (99) -+- * HashAggregate (98) - +- * ColumnarToRow (97) - +- CometColumnarExchange (96) - +- RowToColumnar (95) - +- * HashAggregate (94) - +- * Expand (93) - +- Union (92) - :- * Project (34) - : +- * BroadcastHashJoin LeftOuter BuildRight (33) - : :- * HashAggregate (17) - : : +- * ColumnarToRow (16) - : : +- CometColumnarExchange (15) - : : +- RowToColumnar (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (32) - : +- * HashAggregate (31) - : +- * ColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- RowToColumnar (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * ColumnarToRow (20) - : : : +- CometFilter (19) - : : : +- CometScan parquet spark_catalog.default.store_returns (18) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - :- * Project (57) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (56) - : :- BroadcastExchange (45) - : : +- * HashAggregate (44) - : : +- * ColumnarToRow (43) - : : +- CometColumnarExchange (42) - : : +- RowToColumnar (41) - : : +- * HashAggregate (40) - : : +- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * ColumnarToRow (36) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (35) - : : +- ReusedExchange (37) - : +- * HashAggregate (55) - : +- * ColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- RowToColumnar (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * ColumnarToRow (47) - : : +- CometScan parquet spark_catalog.default.catalog_returns (46) - : +- ReusedExchange (48) - +- * Project (91) - +- * BroadcastHashJoin LeftOuter BuildRight (90) - :- * HashAggregate (74) - : +- * ColumnarToRow (73) - : +- CometColumnarExchange (72) - : +- RowToColumnar (71) - : +- * HashAggregate (70) - : +- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * ColumnarToRow (60) - : : : +- CometFilter (59) - : : : +- CometScan parquet spark_catalog.default.web_sales (58) - : : +- ReusedExchange (61) - : +- BroadcastExchange (67) - : +- * ColumnarToRow (66) - : +- CometFilter (65) - : +- CometScan parquet spark_catalog.default.web_page (64) - +- BroadcastExchange (89) - +- * HashAggregate (88) - +- * ColumnarToRow (87) - +- CometColumnarExchange (86) - +- RowToColumnar (85) - +- * HashAggregate (84) - +- * Project (83) - +- * BroadcastHashJoin Inner BuildRight (82) - :- * Project (80) - : +- * BroadcastHashJoin Inner BuildRight (79) - : :- * ColumnarToRow (77) - : : +- CometFilter (76) - : : +- CometScan parquet spark_catalog.default.web_returns (75) - : +- ReusedExchange (78) - +- ReusedExchange (81) +TakeOrderedAndProject (85) ++- * HashAggregate (84) + +- Exchange (83) + +- * HashAggregate (82) + +- * Expand (81) + +- Union (80) + :- * Project (30) + : +- * BroadcastHashJoin LeftOuter BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.store_returns (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + :- * Project (49) + : +- * BroadcastNestedLoopJoin Inner BuildLeft (48) + : :- BroadcastExchange (39) + : : +- * HashAggregate (38) + : : +- Exchange (37) + : : +- * HashAggregate (36) + : : +- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * ColumnarToRow (32) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : +- ReusedExchange (33) + : +- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * ColumnarToRow (41) + : : +- CometScan parquet spark_catalog.default.catalog_returns (40) + : +- ReusedExchange (42) + +- * Project (79) + +- * BroadcastHashJoin LeftOuter BuildRight (78) + :- * HashAggregate (64) + : +- Exchange (63) + : +- * HashAggregate (62) + : +- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * Project (55) + : : +- * BroadcastHashJoin Inner BuildRight (54) + : : :- * ColumnarToRow (52) + : : : +- CometFilter (51) + : : : +- CometScan parquet spark_catalog.default.web_sales (50) + : : +- ReusedExchange (53) + : +- BroadcastExchange (59) + : +- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_page (56) + +- BroadcastExchange (77) + +- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- * Project (73) + +- * BroadcastHashJoin Inner BuildRight (72) + :- * Project (70) + : +- * BroadcastHashJoin Inner BuildRight (69) + : :- * ColumnarToRow (67) + : : +- CometFilter (66) + : : +- CometScan parquet spark_catalog.default.web_returns (65) + : +- ReusedExchange (68) + +- ReusedExchange (71) (unknown) Scan parquet spark_catalog.default.store_sales @@ -115,7 +101,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 3] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 104] +(4) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -163,17 +149,11 @@ Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(Un Aggregate Attributes [2]: [sum#8, sum#9] Results [3]: [s_store_sk#7, sum#10, sum#11] -(14) RowToColumnar +(14) Exchange Input [3]: [s_store_sk#7, sum#10, sum#11] +Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] - -(17) HashAggregate [codegen id : 8] +(15) HashAggregate [codegen id : 8] Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] @@ -188,74 +168,68 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Condition : isnotnull(sr_store_sk#16) -(20) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 6] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -(21) ReusedExchange [Reuses operator id: 104] +(19) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#21] -(22) BroadcastHashJoin [codegen id : 6] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#19] Right keys [1]: [d_date_sk#21] Join type: Inner Join condition: None -(23) Project [codegen id : 6] +(21) Project [codegen id : 6] Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] -(24) ReusedExchange [Reuses operator id: 10] +(22) ReusedExchange [Reuses operator id: 10] Output [1]: [s_store_sk#22] -(25) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#16] Right keys [1]: [s_store_sk#22] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -(27) HashAggregate [codegen id : 6] +(25) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] Keys [1]: [s_store_sk#22] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] Aggregate Attributes [2]: [sum#23, sum#24] Results [3]: [s_store_sk#22, sum#25, sum#26] -(28) RowToColumnar -Input [3]: [s_store_sk#22, sum#25, sum#26] - -(29) CometColumnarExchange -Input [3]: [s_store_sk#22, sum#25, sum#26] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) ColumnarToRow [codegen id : 7] +(26) Exchange Input [3]: [s_store_sk#22, sum#25, sum#26] +Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(31) HashAggregate [codegen id : 7] +(27) HashAggregate [codegen id : 7] Input [3]: [s_store_sk#22, sum#25, sum#26] Keys [1]: [s_store_sk#22] Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] -(32) BroadcastExchange +(28) BroadcastExchange Input [3]: [s_store_sk#22, returns#29, profit_loss#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 8] +(29) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#7] Right keys [1]: [s_store_sk#22] Join type: LeftOuter Join condition: None -(34) Project [codegen id : 8] +(30) Project [codegen id : 8] Output [5]: [sales#14, coalesce(returns#29, 0.00) AS returns#31, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#32, store channel AS channel#33, s_store_sk#7 AS id#34] Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] @@ -266,47 +240,41 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 10] +(32) ColumnarToRow [codegen id : 10] Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] -(37) ReusedExchange [Reuses operator id: 104] +(33) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#40] -(38) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#38] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(39) Project [codegen id : 10] +(35) Project [codegen id : 10] Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] -(40) HashAggregate [codegen id : 10] +(36) HashAggregate [codegen id : 10] Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] Keys [1]: [cs_call_center_sk#35] Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] Aggregate Attributes [2]: [sum#41, sum#42] Results [3]: [cs_call_center_sk#35, sum#43, sum#44] -(41) RowToColumnar -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(42) CometColumnarExchange +(37) Exchange Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(43) ColumnarToRow [codegen id : 11] -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(44) HashAggregate [codegen id : 11] +(38) HashAggregate [codegen id : 11] Input [3]: [cs_call_center_sk#35, sum#43, sum#44] Keys [1]: [cs_call_center_sk#35] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] -(45) BroadcastExchange +(39) BroadcastExchange Input [3]: [cs_call_center_sk#35, sales#47, profit#48] Arguments: IdentityBroadcastMode, [plan_id=6] @@ -317,51 +285,45 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 13] +(41) ColumnarToRow [codegen id : 13] Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -(48) ReusedExchange [Reuses operator id: 104] +(42) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#53] -(49) BroadcastHashJoin [codegen id : 13] +(43) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cr_returned_date_sk#51] Right keys [1]: [d_date_sk#53] Join type: Inner Join condition: None -(50) Project [codegen id : 13] +(44) Project [codegen id : 13] Output [2]: [cr_return_amount#49, cr_net_loss#50] Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] -(51) HashAggregate [codegen id : 13] +(45) HashAggregate [codegen id : 13] Input [2]: [cr_return_amount#49, cr_net_loss#50] Keys: [] Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] Aggregate Attributes [2]: [sum#54, sum#55] Results [2]: [sum#56, sum#57] -(52) RowToColumnar -Input [2]: [sum#56, sum#57] - -(53) CometColumnarExchange +(46) Exchange Input [2]: [sum#56, sum#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(54) ColumnarToRow -Input [2]: [sum#56, sum#57] - -(55) HashAggregate +(47) HashAggregate Input [2]: [sum#56, sum#57] Keys: [] Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] -(56) BroadcastNestedLoopJoin [codegen id : 14] +(48) BroadcastNestedLoopJoin [codegen id : 14] Join type: Inner Join condition: None -(57) Project [codegen id : 14] +(49) Project [codegen id : 14] Output [5]: [sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#62, catalog channel AS channel#63, cs_call_center_sk#35 AS id#64] Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] @@ -373,23 +335,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(59) CometFilter +(51) CometFilter Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] Condition : isnotnull(ws_web_page_sk#65) -(60) ColumnarToRow [codegen id : 17] +(52) ColumnarToRow [codegen id : 17] Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -(61) ReusedExchange [Reuses operator id: 104] +(53) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#70] -(62) BroadcastHashJoin [codegen id : 17] +(54) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_sold_date_sk#68] Right keys [1]: [d_date_sk#70] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(55) Project [codegen id : 17] Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] @@ -400,45 +362,39 @@ Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(65) CometFilter +(57) CometFilter Input [1]: [wp_web_page_sk#71] Condition : isnotnull(wp_web_page_sk#71) -(66) ColumnarToRow [codegen id : 16] +(58) ColumnarToRow [codegen id : 16] Input [1]: [wp_web_page_sk#71] -(67) BroadcastExchange +(59) BroadcastExchange Input [1]: [wp_web_page_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(68) BroadcastHashJoin [codegen id : 17] +(60) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_web_page_sk#65] Right keys [1]: [wp_web_page_sk#71] Join type: Inner Join condition: None -(69) Project [codegen id : 17] +(61) Project [codegen id : 17] Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -(70) HashAggregate [codegen id : 17] +(62) HashAggregate [codegen id : 17] Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] Keys [1]: [wp_web_page_sk#71] Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] Aggregate Attributes [2]: [sum#72, sum#73] Results [3]: [wp_web_page_sk#71, sum#74, sum#75] -(71) RowToColumnar -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(72) CometColumnarExchange +(63) Exchange Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(73) ColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(74) HashAggregate [codegen id : 22] +(64) HashAggregate [codegen id : 22] Input [3]: [wp_web_page_sk#71, sum#74, sum#75] Keys [1]: [wp_web_page_sk#71] Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] @@ -453,119 +409,107 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(76) CometFilter +(66) CometFilter Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] Condition : isnotnull(wr_web_page_sk#80) -(77) ColumnarToRow [codegen id : 20] +(67) ColumnarToRow [codegen id : 20] Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] -(78) ReusedExchange [Reuses operator id: 104] +(68) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#85] -(79) BroadcastHashJoin [codegen id : 20] +(69) BroadcastHashJoin [codegen id : 20] Left keys [1]: [wr_returned_date_sk#83] Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None -(80) Project [codegen id : 20] +(70) Project [codegen id : 20] Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] -(81) ReusedExchange [Reuses operator id: 67] +(71) ReusedExchange [Reuses operator id: 59] Output [1]: [wp_web_page_sk#86] -(82) BroadcastHashJoin [codegen id : 20] +(72) BroadcastHashJoin [codegen id : 20] Left keys [1]: [wr_web_page_sk#80] Right keys [1]: [wp_web_page_sk#86] Join type: Inner Join condition: None -(83) Project [codegen id : 20] +(73) Project [codegen id : 20] Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -(84) HashAggregate [codegen id : 20] +(74) HashAggregate [codegen id : 20] Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] Keys [1]: [wp_web_page_sk#86] Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] Aggregate Attributes [2]: [sum#87, sum#88] Results [3]: [wp_web_page_sk#86, sum#89, sum#90] -(85) RowToColumnar -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(86) CometColumnarExchange -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(87) ColumnarToRow [codegen id : 21] +(75) Exchange Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(88) HashAggregate [codegen id : 21] +(76) HashAggregate [codegen id : 21] Input [3]: [wp_web_page_sk#86, sum#89, sum#90] Keys [1]: [wp_web_page_sk#86] Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] -(89) BroadcastExchange +(77) BroadcastExchange Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(90) BroadcastHashJoin [codegen id : 22] +(78) BroadcastHashJoin [codegen id : 22] Left keys [1]: [wp_web_page_sk#71] Right keys [1]: [wp_web_page_sk#86] Join type: LeftOuter Join condition: None -(91) Project [codegen id : 22] +(79) Project [codegen id : 22] Output [5]: [sales#78, coalesce(returns#93, 0.00) AS returns#95, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#96, web channel AS channel#97, wp_web_page_sk#71 AS id#98] Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] -(92) Union +(80) Union -(93) Expand [codegen id : 23] +(81) Expand [codegen id : 23] Input [5]: [sales#14, returns#31, profit#32, channel#33, id#34] Arguments: [[sales#14, returns#31, profit#32, channel#33, id#34, 0], [sales#14, returns#31, profit#32, channel#33, null, 1], [sales#14, returns#31, profit#32, null, null, 3]], [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] -(94) HashAggregate [codegen id : 23] +(82) HashAggregate [codegen id : 23] Input [6]: [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] Keys [3]: [channel#99, id#100, spark_grouping_id#101] Functions [3]: [partial_sum(sales#14), partial_sum(returns#31), partial_sum(profit#32)] Aggregate Attributes [6]: [sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] Results [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -(95) RowToColumnar -Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] - -(96) CometColumnarExchange -Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Arguments: hashpartitioning(channel#99, id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(97) ColumnarToRow [codegen id : 24] +(83) Exchange Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Arguments: hashpartitioning(channel#99, id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(98) HashAggregate [codegen id : 24] +(84) HashAggregate [codegen id : 24] Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] Keys [3]: [channel#99, id#100, spark_grouping_id#101] Functions [3]: [sum(sales#14), sum(returns#31), sum(profit#32)] Aggregate Attributes [3]: [sum(sales#14)#114, sum(returns#31)#115, sum(profit#32)#116] Results [5]: [channel#99, id#100, sum(sales#14)#114 AS sales#117, sum(returns#31)#115 AS returns#118, sum(profit#32)#116 AS profit#119] -(99) TakeOrderedAndProject +(85) TakeOrderedAndProject Input [5]: [channel#99, id#100, sales#117, returns#118, profit#119] Arguments: 100, [channel#99 ASC NULLS FIRST, id#100 ASC NULLS FIRST], [channel#99, id#100, sales#117, returns#118, profit#119] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (104) -+- * ColumnarToRow (103) - +- CometProject (102) - +- CometFilter (101) - +- CometScan parquet spark_catalog.default.date_dim (100) +BroadcastExchange (90) ++- * ColumnarToRow (89) + +- CometProject (88) + +- CometFilter (87) + +- CometScan parquet spark_catalog.default.date_dim (86) (unknown) Scan parquet spark_catalog.default.date_dim @@ -575,29 +519,29 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] ReadSchema: struct -(101) CometFilter +(87) CometFilter Input [2]: [d_date_sk#6, d_date#120] Condition : (((isnotnull(d_date#120) AND (d_date#120 >= 2000-08-03)) AND (d_date#120 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) -(102) CometProject +(88) CometProject Input [2]: [d_date_sk#6, d_date#120] Arguments: [d_date_sk#6], [d_date_sk#6] -(103) ColumnarToRow [codegen id : 1] +(89) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(104) BroadcastExchange +(90) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 46 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 40 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 75 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 65 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index f0ab076bf8..d6693067f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -1,157 +1,143 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (24) 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - RowToColumnar - WholeStageCodegen (23) - 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 (8) - Project [sales,returns,profit,profit_loss,s_store_sk] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan 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) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #6 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + InputAdapter + Exchange [channel,id,spark_grouping_id] #1 + WholeStageCodegen (23) + 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 (8) + Project [sales,returns,profit,profit_loss,s_store_sk] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + InputAdapter + Exchange [s_store_sk] #2 + WholeStageCodegen (3) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan 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) ColumnarToRow InputAdapter - CometFilter [sr_store_sk] - CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [s_store_sk] #4 - WholeStageCodegen (14) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - ColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometColumnarExchange [cs_call_center_sk] #8 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange #9 - RowToColumnar - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) - Project [sales,returns,profit,profit_loss,wp_web_page_sk] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - ColumnarToRow + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - CometColumnarExchange [wp_web_page_sk] #10 - RowToColumnar - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_web_page_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Exchange [s_store_sk] #6 + WholeStageCodegen (6) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [sr_store_sk] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [s_store_sk] #4 + WholeStageCodegen (14) + Project [sales,returns,profit,profit_loss,cs_call_center_sk] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] + InputAdapter + Exchange [cs_call_center_sk] #8 + WholeStageCodegen (10) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #11 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - ColumnarToRow + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + Exchange #9 + WholeStageCodegen (13) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - CometColumnarExchange [wp_web_page_sk] #13 - RowToColumnar - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [wr_web_page_sk] - CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [wp_web_page_sk] #11 + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (22) + Project [sales,returns,profit,profit_loss,wp_web_page_sk] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + InputAdapter + Exchange [wp_web_page_sk] #10 + WholeStageCodegen (17) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (21) + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + Exchange [wp_web_page_sk] #13 + WholeStageCodegen (20) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_web_page_sk] + CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [wp_web_page_sk] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index f633020bc1..d9e4b80d83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -1,77 +1,74 @@ == Physical Plan == -TakeOrderedAndProject (73) -+- * Project (72) - +- * SortMergeJoin Inner (71) - :- * Project (47) - : +- * SortMergeJoin Inner (46) - : :- * Sort (22) - : : +- * HashAggregate (21) - : : +- * ColumnarToRow (20) - : : +- CometColumnarExchange (19) - : : +- RowToColumnar (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometColumnarExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometColumnarExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (14) - : +- * Sort (45) - : +- * Filter (44) - : +- * HashAggregate (43) - : +- * ColumnarToRow (42) - : +- CometColumnarExchange (41) - : +- RowToColumnar (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * ColumnarToRow (35) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometSortMergeJoin (32) - : : :- CometSort (26) - : : : +- CometColumnarExchange (25) - : : : +- CometFilter (24) - : : : +- CometScan parquet spark_catalog.default.web_sales (23) - : : +- CometSort (31) - : : +- CometColumnarExchange (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (36) - +- * Sort (70) - +- * Filter (69) - +- * HashAggregate (68) - +- * ColumnarToRow (67) - +- CometColumnarExchange (66) - +- RowToColumnar (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) - : +- CometProject (59) - : +- CometFilter (58) - : +- CometSortMergeJoin (57) - : :- CometSort (51) - : : +- CometColumnarExchange (50) - : : +- CometFilter (49) - : : +- CometScan parquet spark_catalog.default.catalog_sales (48) - : +- CometSort (56) - : +- CometColumnarExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan parquet spark_catalog.default.catalog_returns (52) - +- ReusedExchange (61) +TakeOrderedAndProject (70) ++- * Project (69) + +- * SortMergeJoin Inner (68) + :- * Project (45) + : +- * SortMergeJoin Inner (44) + : :- * Sort (21) + : : +- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Project (14) + : : : +- * Filter (13) + : : : +- * SortMergeJoin LeftOuter (12) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- * Sort (11) + : : : +- Exchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometProject (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : +- ReusedExchange (15) + : +- * Sort (43) + : +- * Filter (42) + : +- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (35) + : : +- * Filter (34) + : : +- * SortMergeJoin LeftOuter (33) + : : :- * Sort (26) + : : : +- Exchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.web_returns (27) + : +- ReusedExchange (36) + +- * Sort (67) + +- * Filter (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) + :- * Project (59) + : +- * Filter (58) + : +- * SortMergeJoin LeftOuter (57) + : :- * Sort (50) + : : +- Exchange (49) + : : +- * ColumnarToRow (48) + : : +- CometFilter (47) + : : +- CometScan parquet spark_catalog.default.catalog_sales (46) + : +- * Sort (56) + : +- Exchange (55) + : +- * ColumnarToRow (54) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.catalog_returns (51) + +- ReusedExchange (60) (unknown) Scan parquet spark_catalog.default.store_sales @@ -86,13 +83,16 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) -(7) CometProject +(8) CometProject Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] -(8) CometColumnarExchange +(9) ColumnarToRow [codegen id : 3] +Input [2]: [sr_item_sk#9, sr_ticket_number#10] + +(10) Exchange Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(9) CometSort +(11) Sort [codegen id : 4] Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] +Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter +(12) SortMergeJoin [codegen id : 6] +Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] +Join type: LeftOuter +Join condition: None -(11) CometFilter +(13) Filter [codegen id : 6] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Condition : isnull(sr_ticket_number#10) -(12) CometProject +(14) Project [codegen id : 6] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(14) ReusedExchange [Reuses operator id: 77] +(15) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#12, d_year#13] -(15) BroadcastHashJoin [codegen id : 2] +(16) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(16) Project [codegen id : 2] +(17) Project [codegen id : 6] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -(17) HashAggregate [codegen id : 2] +(18) HashAggregate [codegen id : 6] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum#14, sum#15, sum#16] Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(18) RowToColumnar -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] - -(19) CometColumnarExchange +(19) Exchange Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) ColumnarToRow [codegen id : 3] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] - -(21) HashAggregate [codegen id : 3] +(20) HashAggregate [codegen id : 7] Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(22) Sort [codegen id : 3] +(21) Sort [codegen id : 7] Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 @@ -182,17 +177,20 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(24) CometFilter +(23) CometFilter Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) -(25) CometColumnarExchange +(24) ColumnarToRow [codegen id : 8] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] + +(25) Exchange Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(26) CometSort +(26) Sort [codegen id : 9] Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] +Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_returns Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] @@ -209,82 +207,77 @@ Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] -(30) CometColumnarExchange +(30) ColumnarToRow [codegen id : 10] Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(31) CometSort +(31) Exchange Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(32) CometSortMergeJoin -Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Right output [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter +(32) Sort [codegen id : 11] +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 + +(33) SortMergeJoin [codegen id : 13] +Left keys [2]: [ws_order_number#29, ws_item_sk#27] +Right keys [2]: [wr_order_number#36, wr_item_sk#35] +Join type: LeftOuter +Join condition: None -(33) CometFilter +(34) Filter [codegen id : 13] Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] Condition : isnull(wr_order_number#36) -(34) CometProject +(35) Project [codegen id : 13] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -(35) ColumnarToRow [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] - -(36) ReusedExchange [Reuses operator id: 77] +(36) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#38, d_year#39] -(37) BroadcastHashJoin [codegen id : 5] +(37) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#33] Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(38) Project [codegen id : 5] +(38) Project [codegen id : 13] Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] -(39) HashAggregate [codegen id : 5] +(39) HashAggregate [codegen id : 13] Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum#40, sum#41, sum#42] Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -(40) RowToColumnar -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(41) CometColumnarExchange -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) ColumnarToRow [codegen id : 6] +(40) Exchange Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(43) HashAggregate [codegen id : 6] +(41) HashAggregate [codegen id : 14] Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] -(44) Filter [codegen id : 6] +(42) Filter [codegen id : 14] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Condition : (coalesce(ws_qty#51, 0) > 0) -(45) Sort [codegen id : 6] +(43) Sort [codegen id : 14] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 -(46) SortMergeJoin [codegen id : 7] +(44) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] Join type: Inner Join condition: None -(47) Project [codegen id : 7] +(45) Project [codegen id : 15] Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] @@ -296,17 +289,20 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(49) CometFilter +(47) CometFilter Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) -(50) CometColumnarExchange +(48) ColumnarToRow [codegen id : 16] +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] + +(49) Exchange Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(51) CometSort +(50) Sort [codegen id : 17] Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] +Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] @@ -315,104 +311,99 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(53) CometFilter +(52) CometFilter Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) -(54) CometProject +(53) CometProject Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] -(55) CometColumnarExchange +(54) ColumnarToRow [codegen id : 18] Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(56) CometSort +(55) Exchange Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(57) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Right output [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter +(56) Sort [codegen id : 19] +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 -(58) CometFilter +(57) SortMergeJoin [codegen id : 21] +Left keys [2]: [cs_order_number#56, cs_item_sk#55] +Right keys [2]: [cr_order_number#63, cr_item_sk#62] +Join type: LeftOuter +Join condition: None + +(58) Filter [codegen id : 21] Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] Condition : isnull(cr_order_number#63) -(59) CometProject +(59) Project [codegen id : 21] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(60) ColumnarToRow [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] - -(61) ReusedExchange [Reuses operator id: 77] +(60) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#65, d_year#66] -(62) BroadcastHashJoin [codegen id : 9] +(61) BroadcastHashJoin [codegen id : 21] Left keys [1]: [cs_sold_date_sk#60] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(63) Project [codegen id : 9] +(62) Project [codegen id : 21] Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] -(64) HashAggregate [codegen id : 9] +(63) HashAggregate [codegen id : 21] Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum#67, sum#68, sum#69] Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -(65) RowToColumnar -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] - -(66) CometColumnarExchange -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(67) ColumnarToRow [codegen id : 10] +(64) Exchange Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(68) HashAggregate [codegen id : 10] +(65) HashAggregate [codegen id : 22] Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] -(69) Filter [codegen id : 10] +(66) Filter [codegen id : 22] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Condition : (coalesce(cs_qty#78, 0) > 0) -(70) Sort [codegen id : 10] +(67) Sort [codegen id : 22] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 -(71) SortMergeJoin [codegen id : 11] +(68) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] Join type: Inner Join condition: None -(72) Project [codegen id : 11] +(69) Project [codegen id : 23] Output [12]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -(73) TakeOrderedAndProject +(70) TakeOrderedAndProject Input [12]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] Arguments: 100, [ratio#81 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (77) -+- * ColumnarToRow (76) - +- CometFilter (75) - +- CometScan parquet spark_catalog.default.date_dim (74) +BroadcastExchange (74) ++- * ColumnarToRow (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) (unknown) Scan parquet spark_catalog.default.date_dim @@ -422,19 +413,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(75) CometFilter +(72) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(76) ColumnarToRow [codegen id : 1] +(73) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(77) BroadcastExchange +(74) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 48 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index cc0372b8ad..280687e301 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -1,103 +1,127 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (11) + WholeStageCodegen (23) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] InputAdapter - WholeStageCodegen (7) + WholeStageCodegen (15) Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] InputAdapter - WholeStageCodegen (3) + WholeStageCodegen (7) Sort [ss_sold_year,ss_item_sk,ss_customer_sk] HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow + InputAdapter + Exchange [d_year,ss_item_sk,ss_customer_sk] #1 + WholeStageCodegen (6) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + Filter [sr_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [sr_ticket_number] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (14) Sort [ws_sold_year,ws_item_sk,ws_customer_sk] Filter [ws_qty] HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow + InputAdapter + Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + WholeStageCodegen (13) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + Filter [wr_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [wr_order_number] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #6 - CometFilter [ws_item_sk,ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_order_number,wr_item_sk] - CometColumnarExchange [wr_order_number,wr_item_sk] #7 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + WholeStageCodegen (9) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #6 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (11) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #7 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter - WholeStageCodegen (10) + WholeStageCodegen (22) Sort [cs_sold_year,cs_item_sk,cs_customer_sk] Filter [cs_qty] HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow + InputAdapter + Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 + WholeStageCodegen (21) + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + Filter [cr_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (17) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #9 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cr_order_number] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #9 - CometFilter [cs_item_sk,cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_order_number,cr_item_sk] - CometColumnarExchange [cr_order_number,cr_item_sk] #10 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + WholeStageCodegen (19) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #10 + WholeStageCodegen (18) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt index 9bb7383bf2..667c05e1c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt @@ -1,36 +1,34 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * HashAggregate (25) - : +- * ColumnarToRow (24) - : +- CometColumnarExchange (23) - : +- RowToColumnar (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.customer (26) +TakeOrderedAndProject (30) ++- * Project (29) + +- * BroadcastHashJoin Inner BuildRight (28) + :- * HashAggregate (23) + : +- Exchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (27) + +- * ColumnarToRow (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.customer (24) (unknown) Scan parquet spark_catalog.default.store_sales @@ -48,7 +46,7 @@ Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnu (3) ColumnarToRow [codegen id : 4] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -(4) ReusedExchange [Reuses operator id: 37] +(4) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#10] (5) BroadcastHashJoin [codegen id : 4] @@ -132,17 +130,11 @@ Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(Unscale Aggregate Attributes [2]: [sum#17, sum#18] Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -(22) RowToColumnar +(22) Exchange Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(23) CometColumnarExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) ColumnarToRow [codegen id : 6] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] - -(25) HashAggregate [codegen id : 6] +(23) HashAggregate [codegen id : 6] Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] @@ -156,39 +148,39 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(27) CometFilter +(25) CometFilter Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] Condition : isnotnull(c_customer_sk#25) -(28) ColumnarToRow [codegen id : 5] +(26) ColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -(29) BroadcastExchange +(27) BroadcastExchange Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(30) BroadcastHashJoin [codegen id : 6] +(28) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(29) Project [codegen id : 6] Output [7]: [c_last_name#27, c_first_name#26, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -(32) TakeOrderedAndProject +(30) TakeOrderedAndProject Input [7]: [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] Arguments: 100, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) (unknown) Scan parquet spark_catalog.default.date_dim @@ -198,18 +190,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(32) CometFilter Input [3]: [d_date_sk#10, d_year#29, d_dow#30] Condition : (((isnotnull(d_dow#30) AND (d_dow#30 = 1)) AND d_year#29 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(35) CometProject +(33) CometProject Input [3]: [d_date_sk#10, d_year#29, d_dow#30] Arguments: [d_date_sk#10], [d_date_sk#10] -(36) ColumnarToRow [codegen id : 1] +(34) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(37) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt index e4fa69b477..4c05c449c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt @@ -3,48 +3,46 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dow,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometProject [s_store_sk,s_city] + CometFilter [s_number_employees,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index 621905e5af..f54999ff58 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -1,53 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * HashAggregate (48) - +- * ColumnarToRow (47) - +- CometColumnarExchange (46) - +- RowToColumnar (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (41) - +- * ColumnarToRow (40) - +- CometHashAggregate (39) - +- CometColumnarExchange (38) - +- RowToColumnar (37) - +- * HashAggregate (36) - +- * BroadcastHashJoin LeftSemi BuildRight (35) - :- * ColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.customer_address (13) - +- BroadcastExchange (34) - +- * Project (33) - +- * Filter (32) - +- * HashAggregate (31) - +- * ColumnarToRow (30) - +- CometColumnarExchange (29) - +- RowToColumnar (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * ColumnarToRow (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.customer_address (17) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan parquet spark_catalog.default.customer (20) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.store (7) + +- BroadcastExchange (37) + +- * HashAggregate (36) + +- Exchange (35) + +- * HashAggregate (34) + +- * BroadcastHashJoin LeftSemi BuildRight (33) + :- * ColumnarToRow (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.customer_address (13) + +- BroadcastExchange (32) + +- * Project (31) + +- * Filter (30) + +- * HashAggregate (29) + +- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * ColumnarToRow (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.customer_address (17) + +- BroadcastExchange (24) + +- * ColumnarToRow (23) + +- CometProject (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.customer (20) (unknown) Scan parquet spark_catalog.default.store_sales @@ -65,7 +59,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 8] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 54] +(4) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 8] @@ -176,113 +170,97 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] Results [2]: [ca_zip#12, count#16] -(28) RowToColumnar +(28) Exchange Input [2]: [ca_zip#12, count#16] +Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(29) CometColumnarExchange -Input [2]: [ca_zip#12, count#16] -Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) ColumnarToRow [codegen id : 5] -Input [2]: [ca_zip#12, count#16] - -(31) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 5] Input [2]: [ca_zip#12, count#16] Keys [1]: [ca_zip#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [substr(ca_zip#12, 1, 5) AS ca_zip#18, count(1)#17 AS cnt#19] -(32) Filter [codegen id : 5] +(30) Filter [codegen id : 5] Input [2]: [ca_zip#18, cnt#19] Condition : (cnt#19 > 10) -(33) Project [codegen id : 5] +(31) Project [codegen id : 5] Output [1]: [ca_zip#18] Input [2]: [ca_zip#18, cnt#19] -(34) BroadcastExchange +(32) BroadcastExchange Input [1]: [ca_zip#18] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [plan_id=4] -(35) BroadcastHashJoin [codegen id : 6] +(33) BroadcastHashJoin [codegen id : 6] Left keys [2]: [coalesce(ca_zip#10, ), isnull(ca_zip#10)] Right keys [2]: [coalesce(ca_zip#18, ), isnull(ca_zip#18)] Join type: LeftSemi Join condition: None -(36) HashAggregate [codegen id : 6] +(34) HashAggregate [codegen id : 6] Input [1]: [ca_zip#10] Keys [1]: [ca_zip#10] Functions: [] Aggregate Attributes: [] Results [1]: [ca_zip#10] -(37) RowToColumnar +(35) Exchange Input [1]: [ca_zip#10] +Arguments: hashpartitioning(ca_zip#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) CometColumnarExchange -Input [1]: [ca_zip#10] -Arguments: hashpartitioning(ca_zip#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(39) CometHashAggregate +(36) HashAggregate [codegen id : 7] Input [1]: [ca_zip#10] Keys [1]: [ca_zip#10] Functions: [] +Aggregate Attributes: [] +Results [1]: [ca_zip#10] -(40) ColumnarToRow [codegen id : 7] -Input [1]: [ca_zip#10] - -(41) BroadcastExchange +(37) BroadcastExchange Input [1]: [ca_zip#10] Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=6] -(42) BroadcastHashJoin [codegen id : 8] +(38) BroadcastHashJoin [codegen id : 8] Left keys [1]: [substr(s_zip#8, 1, 2)] Right keys [1]: [substr(ca_zip#10, 1, 2)] Join type: Inner Join condition: None -(43) Project [codegen id : 8] +(39) Project [codegen id : 8] Output [2]: [ss_net_profit#2, s_store_name#7] Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#10] -(44) HashAggregate [codegen id : 8] +(40) HashAggregate [codegen id : 8] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#20] Results [2]: [s_store_name#7, sum#21] -(45) RowToColumnar -Input [2]: [s_store_name#7, sum#21] - -(46) CometColumnarExchange -Input [2]: [s_store_name#7, sum#21] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(47) ColumnarToRow [codegen id : 9] +(41) Exchange Input [2]: [s_store_name#7, sum#21] +Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(48) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 9] Input [2]: [s_store_name#7, sum#21] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS sum(ss_net_profit)#23] -(49) TakeOrderedAndProject +(43) TakeOrderedAndProject Input [2]: [s_store_name#7, sum(ss_net_profit)#23] Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) (unknown) Scan parquet spark_catalog.default.date_dim @@ -292,18 +270,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter +(45) CometFilter Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2)) AND (d_year#24 = 1998)) AND isnotnull(d_date_sk#5)) -(52) CometProject +(46) CometProject Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] Arguments: [d_date_sk#5], [d_date_sk#5] -(53) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(54) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index df2f062eff..76fa276931 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -1,78 +1,72 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (9) HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_store_name] #1 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [s_store_name,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_store_name] - BroadcastHashJoin [s_zip,ca_zip] - Project [ss_net_profit,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #2 + InputAdapter + Exchange [s_store_name] #1 + WholeStageCodegen (8) + HashAggregate [s_store_name,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_store_name] + BroadcastHashJoin [s_zip,ca_zip] + Project [ss_net_profit,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip] - CometColumnarExchange [ca_zip] #5 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [ca_zip] - BroadcastHashJoin [ca_zip,ca_zip] - ColumnarToRow - InputAdapter - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_zip] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [ca_zip] - Filter [cnt] - HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip] #7 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [ca_zip] [count,count] - Project [ca_zip] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] + CometFilter [s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [ca_zip] + InputAdapter + Exchange [ca_zip] #5 + WholeStageCodegen (6) + HashAggregate [ca_zip] + BroadcastHashJoin [ca_zip,ca_zip] + ColumnarToRow + InputAdapter + CometProject [ca_zip] [ca_zip] + CometFilter [ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_zip] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + Project [ca_zip] + Filter [cnt] + HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] + InputAdapter + Exchange [ca_zip] #7 + WholeStageCodegen (4) + HashAggregate [ca_zip] [count,count] + Project [ca_zip] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [c_current_addr_sk] + CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] 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 9c098dccbe..338f860872 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,116 +1,111 @@ == Physical Plan == -TakeOrderedAndProject (112) -+- * HashAggregate (111) - +- * ColumnarToRow (110) - +- CometColumnarExchange (109) - +- RowToColumnar (108) - +- * HashAggregate (107) - +- * Expand (106) - +- Union (105) - :- * HashAggregate (40) - : +- * ColumnarToRow (39) - : +- CometColumnarExchange (38) - : +- RowToColumnar (37) - : +- * HashAggregate (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * ColumnarToRow (12) - : : : : : +- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometColumnarExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometColumnarExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : : : +- ReusedExchange (13) - : : : +- BroadcastExchange (19) - : : : +- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.store (16) - : : +- BroadcastExchange (26) - : : +- * ColumnarToRow (25) - : : +- CometProject (24) - : : +- CometFilter (23) - : : +- CometScan parquet spark_catalog.default.item (22) - : +- BroadcastExchange (33) - : +- * ColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan parquet spark_catalog.default.promotion (29) - :- * HashAggregate (72) - : +- * ColumnarToRow (71) - : +- CometColumnarExchange (70) - : +- RowToColumnar (69) - : +- * HashAggregate (68) - : +- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * Project (55) - : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : :- * ColumnarToRow (52) - : : : : : +- CometProject (51) - : : : : : +- CometSortMergeJoin (50) - : : : : : :- CometSort (44) - : : : : : : +- CometColumnarExchange (43) - : : : : : : +- CometFilter (42) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) - : : : : : +- CometSort (49) - : : : : : +- CometColumnarExchange (48) - : : : : : +- CometProject (47) - : : : : : +- CometFilter (46) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) - : : : : +- ReusedExchange (53) - : : : +- BroadcastExchange (59) - : : : +- * ColumnarToRow (58) - : : : +- CometFilter (57) - : : : +- CometScan parquet spark_catalog.default.catalog_page (56) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- * HashAggregate (104) - +- * ColumnarToRow (103) - +- CometColumnarExchange (102) - +- RowToColumnar (101) - +- * HashAggregate (100) - +- * Project (99) - +- * BroadcastHashJoin Inner BuildRight (98) - :- * Project (96) - : +- * BroadcastHashJoin Inner BuildRight (95) - : :- * Project (93) - : : +- * BroadcastHashJoin Inner BuildRight (92) - : : :- * Project (87) - : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : :- * ColumnarToRow (84) - : : : : +- CometProject (83) - : : : : +- CometSortMergeJoin (82) - : : : : :- CometSort (76) - : : : : : +- CometColumnarExchange (75) - : : : : : +- CometFilter (74) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (73) - : : : : +- CometSort (81) - : : : : +- CometColumnarExchange (80) - : : : : +- CometProject (79) - : : : : +- CometFilter (78) - : : : : +- CometScan parquet spark_catalog.default.web_returns (77) - : : : +- ReusedExchange (85) - : : +- BroadcastExchange (91) - : : +- * ColumnarToRow (90) - : : +- CometFilter (89) - : : +- CometScan parquet spark_catalog.default.web_site (88) - : +- ReusedExchange (94) - +- ReusedExchange (97) +TakeOrderedAndProject (107) ++- * HashAggregate (106) + +- Exchange (105) + +- * HashAggregate (104) + +- * Expand (103) + +- Union (102) + :- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * Project (16) + : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : :- * Project (13) + : : : : : +- * SortMergeJoin LeftOuter (12) + : : : : : :- * Sort (5) + : : : : : : +- Exchange (4) + : : : : : : +- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- * Sort (11) + : : : : : +- Exchange (10) + : : : : : +- * ColumnarToRow (9) + : : : : : +- CometProject (8) + : : : : : +- CometFilter (7) + : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : : +- ReusedExchange (14) + : : : +- BroadcastExchange (20) + : : : +- * ColumnarToRow (19) + : : : +- CometFilter (18) + : : : +- CometScan parquet spark_catalog.default.store (17) + : : +- BroadcastExchange (27) + : : +- * ColumnarToRow (26) + : : +- CometProject (25) + : : +- CometFilter (24) + : : +- CometScan parquet spark_catalog.default.item (23) + : +- BroadcastExchange (34) + : +- * ColumnarToRow (33) + : +- CometProject (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.promotion (30) + :- * HashAggregate (70) + : +- Exchange (69) + : +- * HashAggregate (68) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (61) + : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : :- * Project (55) + : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : :- * Project (52) + : : : : : +- * SortMergeJoin LeftOuter (51) + : : : : : :- * Sort (44) + : : : : : : +- Exchange (43) + : : : : : : +- * ColumnarToRow (42) + : : : : : : +- CometFilter (41) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) + : : : : : +- * Sort (50) + : : : : : +- Exchange (49) + : : : : : +- * ColumnarToRow (48) + : : : : : +- CometProject (47) + : : : : : +- CometFilter (46) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) + : : : : +- ReusedExchange (53) + : : : +- BroadcastExchange (59) + : : : +- * ColumnarToRow (58) + : : : +- CometFilter (57) + : : : +- CometScan parquet spark_catalog.default.catalog_page (56) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + +- * HashAggregate (101) + +- Exchange (100) + +- * HashAggregate (99) + +- * Project (98) + +- * BroadcastHashJoin Inner BuildRight (97) + :- * Project (95) + : +- * BroadcastHashJoin Inner BuildRight (94) + : :- * Project (92) + : : +- * BroadcastHashJoin Inner BuildRight (91) + : : :- * Project (86) + : : : +- * BroadcastHashJoin Inner BuildRight (85) + : : : :- * Project (83) + : : : : +- * SortMergeJoin LeftOuter (82) + : : : : :- * Sort (75) + : : : : : +- Exchange (74) + : : : : : +- * ColumnarToRow (73) + : : : : : +- CometFilter (72) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) + : : : : +- * Sort (81) + : : : : +- Exchange (80) + : : : : +- * ColumnarToRow (79) + : : : : +- CometProject (78) + : : : : +- CometFilter (77) + : : : : +- CometScan parquet spark_catalog.default.web_returns (76) + : : : +- ReusedExchange (84) + : : +- BroadcastExchange (90) + : : +- * ColumnarToRow (89) + : : +- CometFilter (88) + : : +- CometScan parquet spark_catalog.default.web_site (87) + : +- ReusedExchange (93) + +- ReusedExchange (96) (unknown) Scan parquet spark_catalog.default.store_sales @@ -125,13 +120,16 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) -(7) CometProject +(8) CometProject Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(8) CometColumnarExchange +(9) ColumnarToRow [codegen id : 3] Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(9) CometSort +(10) Exchange Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter +(11) Sort [codegen id : 4] +Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(12) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] +Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] +Join type: LeftOuter +Join condition: None -(12) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(13) Project [codegen id : 9] +Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(13) ReusedExchange [Reuses operator id: 117] +(14) ReusedExchange [Reuses operator id: 112] Output [1]: [d_date_sk#14] -(14) BroadcastHashJoin [codegen id : 5] +(15) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(15) Project [codegen id : 5] +(16) Project [codegen id : 9] Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] @@ -188,24 +187,24 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(17) CometFilter +(18) CometFilter Input [2]: [s_store_sk#15, s_store_id#16] Condition : isnotnull(s_store_sk#15) -(18) ColumnarToRow [codegen id : 2] +(19) ColumnarToRow [codegen id : 6] Input [2]: [s_store_sk#15, s_store_id#16] -(19) BroadcastExchange +(20) BroadcastExchange Input [2]: [s_store_sk#15, s_store_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(20) BroadcastHashJoin [codegen id : 5] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(21) Project [codegen id : 5] +(22) Project [codegen id : 9] Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] @@ -216,28 +215,28 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(24) CometFilter Input [2]: [i_item_sk#17, i_current_price#18] Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) -(24) CometProject +(25) CometProject Input [2]: [i_item_sk#17, i_current_price#18] Arguments: [i_item_sk#17], [i_item_sk#17] -(25) ColumnarToRow [codegen id : 3] +(26) ColumnarToRow [codegen id : 7] Input [1]: [i_item_sk#17] -(26) BroadcastExchange +(27) BroadcastExchange Input [1]: [i_item_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) BroadcastHashJoin [codegen id : 5] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(28) Project [codegen id : 5] +(29) Project [codegen id : 9] Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] @@ -248,49 +247,43 @@ Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] ReadSchema: struct -(30) CometFilter +(31) CometFilter Input [2]: [p_promo_sk#19, p_channel_tv#20] Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) -(31) CometProject +(32) CometProject Input [2]: [p_promo_sk#19, p_channel_tv#20] Arguments: [p_promo_sk#19], [p_promo_sk#19] -(32) ColumnarToRow [codegen id : 4] +(33) ColumnarToRow [codegen id : 8] Input [1]: [p_promo_sk#19] -(33) BroadcastExchange +(34) BroadcastExchange Input [1]: [p_promo_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 5] +(35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_promo_sk#3] Right keys [1]: [p_promo_sk#19] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(36) Project [codegen id : 9] Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] -(36) HashAggregate [codegen id : 5] +(37) HashAggregate [codegen id : 9] Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Keys [1]: [s_store_id#16] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -(37) RowToColumnar -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] - -(38) CometColumnarExchange +(38) Exchange Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(39) ColumnarToRow [codegen id : 6] -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] - -(40) HashAggregate [codegen id : 6] +(39) HashAggregate [codegen id : 10] Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] Keys [1]: [s_store_id#16] 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)))] @@ -305,17 +298,20 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(42) CometFilter +(41) CometFilter Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) -(43) CometColumnarExchange +(42) ColumnarToRow [codegen id : 11] +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] + +(43) Exchange Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(44) CometSort +(44) Sort [codegen id : 12] Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45], [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST] +Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] @@ -332,36 +328,37 @@ Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -(48) CometColumnarExchange +(48) ColumnarToRow [codegen id : 13] +Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] + +(49) Exchange Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(49) CometSort +(50) Sort [codegen id : 14] Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST] +Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 -(50) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Right output [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cs_item_sk#40, cs_order_number#42], [cr_item_sk#47, cr_order_number#48], LeftOuter +(51) SortMergeJoin [codegen id : 19] +Left keys [2]: [cs_item_sk#40, cs_order_number#42] +Right keys [2]: [cr_item_sk#47, cr_order_number#48] +Join type: LeftOuter +Join condition: None -(51) CometProject +(52) Project [codegen id : 19] +Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50], [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -(52) ColumnarToRow [codegen id : 11] -Input [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] - -(53) ReusedExchange [Reuses operator id: 117] +(53) ReusedExchange [Reuses operator id: 112] Output [1]: [d_date_sk#52] -(54) BroadcastHashJoin [codegen id : 11] +(54) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#45] Right keys [1]: [d_date_sk#52] Join type: Inner Join condition: None -(55) Project [codegen id : 11] +(55) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] @@ -376,67 +373,61 @@ ReadSchema: struct Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Condition : isnotnull(cp_catalog_page_sk#53) -(58) ColumnarToRow [codegen id : 8] +(58) ColumnarToRow [codegen id : 16] Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] (59) BroadcastExchange Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 11] +(60) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#39] Right keys [1]: [cp_catalog_page_sk#53] Join type: Inner Join condition: None -(61) Project [codegen id : 11] +(61) Project [codegen id : 19] Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] -(62) ReusedExchange [Reuses operator id: 26] +(62) ReusedExchange [Reuses operator id: 27] Output [1]: [i_item_sk#55] -(63) BroadcastHashJoin [codegen id : 11] +(63) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#40] Right keys [1]: [i_item_sk#55] Join type: Inner Join condition: None -(64) Project [codegen id : 11] +(64) Project [codegen id : 19] Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] -(65) ReusedExchange [Reuses operator id: 33] +(65) ReusedExchange [Reuses operator id: 34] Output [1]: [p_promo_sk#56] -(66) BroadcastHashJoin [codegen id : 11] +(66) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#41] Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(67) Project [codegen id : 11] +(67) Project [codegen id : 19] Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] -(68) HashAggregate [codegen id : 11] +(68) HashAggregate [codegen id : 19] Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -(69) RowToColumnar -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] - -(70) CometColumnarExchange -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(71) ColumnarToRow [codegen id : 12] +(69) Exchange Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(72) HashAggregate [codegen id : 12] +(70) HashAggregate [codegen id : 20] Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] @@ -451,17 +442,20 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(74) CometFilter +(72) CometFilter Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) -(75) CometColumnarExchange +(73) ColumnarToRow [codegen id : 21] +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] + +(74) Exchange Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(76) CometSort +(75) Sort [codegen id : 22] Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81], [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST] +Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] @@ -470,44 +464,45 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(78) CometFilter +(77) CometFilter Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) -(79) CometProject +(78) CometProject Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -(80) CometColumnarExchange +(79) ColumnarToRow [codegen id : 23] Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(81) CometSort +(80) Exchange Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST] +Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(82) CometSortMergeJoin -Left output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Right output [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [ws_item_sk#75, ws_order_number#78], [wr_item_sk#83, wr_order_number#84], LeftOuter +(81) Sort [codegen id : 24] +Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 -(83) CometProject -Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86], [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] +(82) SortMergeJoin [codegen id : 29] +Left keys [2]: [ws_item_sk#75, ws_order_number#78] +Right keys [2]: [wr_item_sk#83, wr_order_number#84] +Join type: LeftOuter +Join condition: None -(84) ColumnarToRow [codegen id : 17] -Input [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] +(83) Project [codegen id : 29] +Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] +Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -(85) ReusedExchange [Reuses operator id: 117] +(84) ReusedExchange [Reuses operator id: 112] Output [1]: [d_date_sk#88] -(86) BroadcastHashJoin [codegen id : 17] +(85) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_sold_date_sk#81] Right keys [1]: [d_date_sk#88] Join type: Inner Join condition: None -(87) Project [codegen id : 17] +(86) Project [codegen id : 29] Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] @@ -518,119 +513,107 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(89) CometFilter +(88) CometFilter Input [2]: [web_site_sk#89, web_site_id#90] Condition : isnotnull(web_site_sk#89) -(90) ColumnarToRow [codegen id : 14] +(89) ColumnarToRow [codegen id : 26] Input [2]: [web_site_sk#89, web_site_id#90] -(91) BroadcastExchange +(90) BroadcastExchange Input [2]: [web_site_sk#89, web_site_id#90] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(92) BroadcastHashJoin [codegen id : 17] +(91) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_web_site_sk#76] Right keys [1]: [web_site_sk#89] Join type: Inner Join condition: None -(93) Project [codegen id : 17] +(92) Project [codegen id : 29] Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] -(94) ReusedExchange [Reuses operator id: 26] +(93) ReusedExchange [Reuses operator id: 27] Output [1]: [i_item_sk#91] -(95) BroadcastHashJoin [codegen id : 17] +(94) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_item_sk#75] Right keys [1]: [i_item_sk#91] Join type: Inner Join condition: None -(96) Project [codegen id : 17] +(95) Project [codegen id : 29] Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] -(97) ReusedExchange [Reuses operator id: 33] +(96) ReusedExchange [Reuses operator id: 34] Output [1]: [p_promo_sk#92] -(98) BroadcastHashJoin [codegen id : 17] +(97) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_promo_sk#77] Right keys [1]: [p_promo_sk#92] Join type: Inner Join condition: None -(99) Project [codegen id : 17] +(98) Project [codegen id : 29] Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] -(100) HashAggregate [codegen id : 17] +(99) HashAggregate [codegen id : 29] Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Keys [1]: [web_site_id#90] Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -(101) RowToColumnar +(100) Exchange Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(102) CometColumnarExchange -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(103) ColumnarToRow [codegen id : 18] -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] - -(104) HashAggregate [codegen id : 18] +(101) HashAggregate [codegen id : 30] Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] Keys [1]: [web_site_id#90] Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#106, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#107, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#108, web channel AS channel#109, concat(web_site, web_site_id#90) AS id#110] -(105) Union +(102) Union -(106) Expand [codegen id : 19] +(103) Expand [codegen id : 31] Input [5]: [sales#34, returns#35, profit#36, channel#37, id#38] Arguments: [[sales#34, returns#35, profit#36, channel#37, id#38, 0], [sales#34, returns#35, profit#36, channel#37, null, 1], [sales#34, returns#35, profit#36, null, null, 3]], [sales#34, returns#35, profit#36, channel#111, id#112, spark_grouping_id#113] -(107) HashAggregate [codegen id : 19] +(104) HashAggregate [codegen id : 31] Input [6]: [sales#34, returns#35, profit#36, channel#111, id#112, spark_grouping_id#113] Keys [3]: [channel#111, id#112, spark_grouping_id#113] Functions [3]: [partial_sum(sales#34), partial_sum(returns#35), partial_sum(profit#36)] Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] Results [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -(108) RowToColumnar -Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(109) CometColumnarExchange -Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Arguments: hashpartitioning(channel#111, id#112, spark_grouping_id#113, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(110) ColumnarToRow [codegen id : 20] +(105) Exchange Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Arguments: hashpartitioning(channel#111, id#112, spark_grouping_id#113, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(111) HashAggregate [codegen id : 20] +(106) HashAggregate [codegen id : 32] Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] Keys [3]: [channel#111, id#112, spark_grouping_id#113] Functions [3]: [sum(sales#34), sum(returns#35), sum(profit#36)] Aggregate Attributes [3]: [sum(sales#34)#126, sum(returns#35)#127, sum(profit#36)#128] Results [5]: [channel#111, id#112, sum(sales#34)#126 AS sales#129, sum(returns#35)#127 AS returns#130, sum(profit#36)#128 AS profit#131] -(112) TakeOrderedAndProject +(107) TakeOrderedAndProject Input [5]: [channel#111, id#112, sales#129, returns#130, profit#131] Arguments: 100, [channel#111 ASC NULLS FIRST, id#112 ASC NULLS FIRST], [channel#111, id#112, sales#129, returns#130, profit#131] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (112) ++- * ColumnarToRow (111) + +- CometProject (110) + +- CometFilter (109) + +- CometScan parquet spark_catalog.default.date_dim (108) (unknown) Scan parquet spark_catalog.default.date_dim @@ -640,23 +623,23 @@ 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 -(114) CometFilter +(109) CometFilter Input [2]: [d_date_sk#14, d_date#132] Condition : (((isnotnull(d_date#132) AND (d_date#132 >= 2000-08-23)) AND (d_date#132 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(115) CometProject +(110) CometProject Input [2]: [d_date_sk#14, d_date#132] Arguments: [d_date_sk#14], [d_date_sk#14] -(116) ColumnarToRow [codegen id : 1] +(111) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(117) BroadcastExchange +(112) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 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 41a1330584..7e257bdc64 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,37 +1,37 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (20) + WholeStageCodegen (32) 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - RowToColumnar - WholeStageCodegen (19) - 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 (6) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #2 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow + InputAdapter + Exchange [channel,id,spark_grouping_id] #1 + WholeStageCodegen (31) + 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 (10) + 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] + InputAdapter + Exchange [s_store_id] #2 + WholeStageCodegen (9) + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [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] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk,ss_ticket_number] InputAdapter - 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_ticket_number,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_ticket_number] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 + Exchange [ss_item_sk,ss_ticket_number] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] CometScan 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 @@ -42,119 +42,141 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number] - CometColumnarExchange [sr_item_sk,sr_ticket_number] #5 + InputAdapter + WholeStageCodegen (4) + Sort [sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_item_sk,sr_ticket_number] #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometFilter [sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) + BroadcastExchange #6 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (12) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #9 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_channel_tv,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (20) + 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] + InputAdapter + Exchange [cp_catalog_page_id] #9 + WholeStageCodegen (19) + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [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] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (12) + Sort [cs_item_sk,cs_order_number] InputAdapter - 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_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometColumnarExchange [cs_item_sk,cs_order_number] #10 + Exchange [cs_item_sk,cs_order_number] #10 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] CometScan 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] - CometColumnarExchange [cr_item_sk,cr_order_number] #11 + InputAdapter + WholeStageCodegen (14) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #11 + WholeStageCodegen (13) + ColumnarToRow + InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] CometFilter [cr_item_sk,cr_order_number] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter - ReusedExchange [i_item_sk] #7 + ReusedExchange [d_date_sk] #4 InputAdapter - ReusedExchange [p_promo_sk] #8 - WholeStageCodegen (18) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #13 - RowToColumnar - WholeStageCodegen (17) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow + BroadcastExchange #12 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + InputAdapter + ReusedExchange [i_item_sk] #7 + InputAdapter + ReusedExchange [p_promo_sk] #8 + WholeStageCodegen (30) + 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] + InputAdapter + Exchange [web_site_id] #13 + WholeStageCodegen (29) + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [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] + SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + InputAdapter + WholeStageCodegen (22) + Sort [ws_item_sk,ws_order_number] InputAdapter - 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_order_number,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_order_number] - CometColumnarExchange [ws_item_sk,ws_order_number] #14 + Exchange [ws_item_sk,ws_order_number] #14 + WholeStageCodegen (21) + ColumnarToRow + InputAdapter CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] CometScan 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] - CometColumnarExchange [wr_item_sk,wr_order_number] #15 + InputAdapter + WholeStageCodegen (24) + Sort [wr_item_sk,wr_order_number] + InputAdapter + Exchange [wr_item_sk,wr_order_number] #15 + WholeStageCodegen (23) + ColumnarToRow + InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] CometFilter [wr_item_sk,wr_order_number] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter - ReusedExchange [i_item_sk] #7 + ReusedExchange [d_date_sk] #4 InputAdapter - ReusedExchange [p_promo_sk] #8 + BroadcastExchange #16 + WholeStageCodegen (26) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + InputAdapter + ReusedExchange [i_item_sk] #7 + InputAdapter + ReusedExchange [p_promo_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index 4b9f725b9d..d8dc396dd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt @@ -1,58 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (47) - : +- * BroadcastHashJoin Inner BuildRight (46) - : :- * Project (41) - : : +- * BroadcastHashJoin Inner BuildRight (40) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * ColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- RowToColumnar (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (39) - : : +- * Filter (38) - : : +- * HashAggregate (37) - : : +- * ColumnarToRow (36) - : : +- CometColumnarExchange (35) - : : +- RowToColumnar (34) - : : +- * HashAggregate (33) - : : +- * HashAggregate (32) - : : +- * ColumnarToRow (31) - : : +- CometColumnarExchange (30) - : : +- RowToColumnar (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.catalog_returns (19) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (45) - : +- * ColumnarToRow (44) - : +- CometFilter (43) - : +- CometScan parquet spark_catalog.default.customer (42) - +- BroadcastExchange (51) - +- * ColumnarToRow (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.customer_address (48) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan parquet spark_catalog.default.customer_address (7) + : : +- BroadcastExchange (33) + : : +- * Filter (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * HashAggregate (28) + : : +- Exchange (27) + : : +- * HashAggregate (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : +- BroadcastExchange (39) + : +- * ColumnarToRow (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.customer (36) + +- BroadcastExchange (45) + +- * ColumnarToRow (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.customer_address (42) (unknown) Scan parquet spark_catalog.default.catalog_returns @@ -70,7 +64,7 @@ Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_custom (3) ColumnarToRow [codegen id : 3] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -(4) ReusedExchange [Reuses operator id: 59] +(4) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -118,24 +112,18 @@ Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum#9] Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] -(14) RowToColumnar +(14) Exchange Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 11] -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] - -(17) HashAggregate [codegen id : 11] +(15) HashAggregate [codegen id : 11] Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] Keys [2]: [cr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#11] Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] -(18) Filter [codegen id : 11] +(16) Filter [codegen id : 11] Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] Condition : isnotnull(ctr_total_return#14) @@ -147,102 +135,90 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(20) CometFilter +(18) CometFilter Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Condition : isnotnull(cr_returning_addr_sk#2) -(21) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 6] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -(22) ReusedExchange [Reuses operator id: 59] +(20) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#6] -(23) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(22) Project [codegen id : 6] Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] -(25) ReusedExchange [Reuses operator id: 10] +(23) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#7, ca_state#8] -(26) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(25) Project [codegen id : 6] Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#8] -(28) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 6] Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] Keys [2]: [cr_returning_customer_sk#1, ca_state#8] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum#16] Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] -(29) RowToColumnar +(27) Exchange Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(30) CometColumnarExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(31) ColumnarToRow [codegen id : 7] -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] - -(32) HashAggregate [codegen id : 7] +(28) HashAggregate [codegen id : 7] Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] Keys [2]: [cr_returning_customer_sk#1, ca_state#8] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#11] Results [2]: [ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] -(33) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 7] Input [2]: [ctr_state#13, ctr_total_return#14] Keys [1]: [ctr_state#13] Functions [1]: [partial_avg(ctr_total_return#14)] Aggregate Attributes [2]: [sum#18, count#19] Results [3]: [ctr_state#13, sum#20, count#21] -(34) RowToColumnar -Input [3]: [ctr_state#13, sum#20, count#21] - -(35) CometColumnarExchange +(30) Exchange Input [3]: [ctr_state#13, sum#20, count#21] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(36) ColumnarToRow [codegen id : 8] -Input [3]: [ctr_state#13, sum#20, count#21] - -(37) HashAggregate [codegen id : 8] +(31) HashAggregate [codegen id : 8] Input [3]: [ctr_state#13, sum#20, count#21] Keys [1]: [ctr_state#13] Functions [1]: [avg(ctr_total_return#14)] Aggregate Attributes [1]: [avg(ctr_total_return#14)#22] Results [2]: [(avg(ctr_total_return#14)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#13 AS ctr_state#13#24] -(38) Filter [codegen id : 8] +(32) Filter [codegen id : 8] Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) -(39) BroadcastExchange +(33) BroadcastExchange Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(40) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#13] Right keys [1]: [ctr_state#13#24] Join type: Inner Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) -(41) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [2]: [ctr_customer_sk#12, ctr_total_return#14] Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] @@ -253,24 +229,24 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(43) CometFilter +(37) CometFilter Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) -(44) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 9] Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] -(45) BroadcastExchange +(39) BroadcastExchange Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 11] +(40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#12] Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(47) Project [codegen id : 11] +(41) Project [codegen id : 11] Output [6]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] @@ -281,39 +257,39 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(49) CometFilter +(43) CometFilter Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] Condition : ((isnotnull(ca_state#38) AND (ca_state#38 = GA)) AND isnotnull(ca_address_sk#31)) -(50) ColumnarToRow [codegen id : 10] +(44) ColumnarToRow [codegen id : 10] Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -(51) BroadcastExchange +(45) BroadcastExchange Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 11] +(46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [c_current_addr_sk#27] Right keys [1]: [ca_address_sk#31] Join type: Inner Join condition: None -(53) Project [codegen id : 11] +(47) Project [codegen id : 11] Output [16]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] Input [18]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -(54) TakeOrderedAndProject +(48) TakeOrderedAndProject Input [16]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] Arguments: 100, [c_customer_id#26 ASC NULLS FIRST, c_salutation#28 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, c_last_name#30 ASC NULLS FIRST, ca_street_number#32 ASC NULLS FIRST, ca_street_name#33 ASC NULLS FIRST, ca_street_type#34 ASC NULLS FIRST, ca_suite_number#35 ASC NULLS FIRST, ca_city#36 ASC NULLS FIRST, ca_county#37 ASC NULLS FIRST, ca_state#38 ASC NULLS FIRST, ca_zip#39 ASC NULLS FIRST, ca_country#40 ASC NULLS FIRST, ca_gmt_offset#41 ASC NULLS FIRST, ca_location_type#42 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (59) -+- * ColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometScan parquet spark_catalog.default.date_dim (55) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) (unknown) Scan parquet spark_catalog.default.date_dim @@ -323,21 +299,21 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(56) CometFilter +(50) CometFilter Input [2]: [d_date_sk#6, d_year#43] Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#6)) -(57) CometProject +(51) CometProject Input [2]: [d_date_sk#6, d_year#43] Arguments: [d_date_sk#6], [d_date_sk#6] -(58) ColumnarToRow [codegen id : 1] +(52) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(59) BroadcastExchange +(53) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index 23c12c9722..da163f0233 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -8,68 +8,62 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 + InputAdapter + Exchange [cr_returning_customer_sk,ca_state] #1 + WholeStageCodegen (3) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 WholeStageCodegen (8) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #5 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [cr_returning_customer_sk,ca_state] #6 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cr_returning_addr_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 + InputAdapter + Exchange [ctr_state] #5 + WholeStageCodegen (7) + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [cr_returning_customer_sk,ca_state] #6 + WholeStageCodegen (6) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cr_returning_addr_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 WholeStageCodegen (9) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt index 694fc26e95..683f7bd940 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt @@ -1,31 +1,29 @@ == Physical Plan == -* ColumnarToRow (27) -+- CometTakeOrderedAndProject (26) - +- CometHashAggregate (25) - +- CometColumnarExchange (24) - +- RowToColumnar (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * ColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan parquet spark_catalog.default.store_sales (16) +TakeOrderedAndProject (25) ++- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildLeft (20) + :- BroadcastExchange (15) + : +- * Project (14) + : +- * BroadcastHashJoin Inner BuildRight (13) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (9) + : : +- * ColumnarToRow (8) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.inventory (5) + : +- ReusedExchange (12) + +- * ColumnarToRow (19) + +- CometProject (18) + +- CometFilter (17) + +- CometScan parquet spark_catalog.default.store_sales (16) (unknown) Scan parquet spark_catalog.default.item @@ -79,7 +77,7 @@ Join condition: None Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_item_sk#6, inv_date_sk#8] -(12) ReusedExchange [Reuses operator id: 32] +(12) ReusedExchange [Reuses operator id: 30] Output [1]: [d_date_sk#10] (13) BroadcastHashJoin [codegen id : 3] @@ -131,33 +129,29 @@ Functions: [] Aggregate Attributes: [] Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(23) RowToColumnar +(23) Exchange Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) CometColumnarExchange -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(25) CometHashAggregate +(24) HashAggregate [codegen id : 5] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] +Aggregate Attributes: [] +Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(26) CometTakeOrderedAndProject -Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#2 ASC NULLS FIRST], output=[i_item_id#2,i_item_desc#3,i_current_price#4]), 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] - -(27) ColumnarToRow [codegen id : 5] +(25) TakeOrderedAndProject Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (32) -+- * ColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan parquet spark_catalog.default.date_dim (28) +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) (unknown) Scan parquet spark_catalog.default.date_dim @@ -167,18 +161,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] ReadSchema: struct -(29) CometFilter +(27) CometFilter Input [2]: [d_date_sk#10, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-05-25)) AND (d_date#13 <= 2000-07-24)) AND isnotnull(d_date_sk#10)) -(30) CometProject +(28) CometProject Input [2]: [d_date_sk#10, d_date#13] Arguments: [d_date_sk#10], [d_date_sk#10] -(31) ColumnarToRow [codegen id : 1] +(29) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(32) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt index 2c652d64fd..0252eb575f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt @@ -1,46 +1,44 @@ -WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + WholeStageCodegen (5) + HashAggregate [i_item_id,i_item_desc,i_current_price] + InputAdapter + Exchange [i_item_id,i_item_desc,i_current_price] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,ss_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (3) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] InputAdapter - ReusedExchange [d_date_sk] #4 - ColumnarToRow - InputAdapter - CometProject [ss_item_sk] - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + ColumnarToRow + InputAdapter + CometProject [ss_item_sk] + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 22a6f8df6e..164ecf8af8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,56 +1,50 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * Project (51) - +- * BroadcastHashJoin Inner BuildRight (50) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * HashAggregate (17) - : : +- * ColumnarToRow (16) - : : +- CometColumnarExchange (15) - : : +- RowToColumnar (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_returns (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (10) - : +- BroadcastExchange (32) - : +- * HashAggregate (31) - : +- * ColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- RowToColumnar (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * ColumnarToRow (20) - : : : +- CometFilter (19) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (49) - +- * HashAggregate (48) - +- * ColumnarToRow (47) - +- CometColumnarExchange (46) - +- RowToColumnar (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * ColumnarToRow (37) - : : +- CometFilter (36) - : : +- CometScan parquet spark_catalog.default.web_returns (35) - : +- ReusedExchange (38) - +- ReusedExchange (41) +TakeOrderedAndProject (46) ++- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_returns (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- ReusedExchange (10) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- BroadcastExchange (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * ColumnarToRow (33) + : : +- CometFilter (32) + : : +- CometScan parquet spark_catalog.default.web_returns (31) + : +- ReusedExchange (34) + +- ReusedExchange (37) (unknown) Scan parquet spark_catalog.default.store_returns @@ -96,7 +90,7 @@ Join condition: None Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#6] -(10) ReusedExchange [Reuses operator id: 68] +(10) ReusedExchange [Reuses operator id: 62] Output [1]: [d_date_sk#7] (11) BroadcastHashJoin [codegen id : 5] @@ -116,17 +110,11 @@ Functions [1]: [partial_sum(sr_return_quantity#2)] Aggregate Attributes [1]: [sum#8] Results [2]: [i_item_id#6, sum#9] -(14) RowToColumnar +(14) Exchange Input [2]: [i_item_id#6, sum#9] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [2]: [i_item_id#6, sum#9] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 18] -Input [2]: [i_item_id#6, sum#9] - -(17) HashAggregate [codegen id : 18] +(15) HashAggregate [codegen id : 18] Input [2]: [i_item_id#6, sum#9] Keys [1]: [i_item_id#6] Functions [1]: [sum(sr_return_quantity#2)] @@ -141,74 +129,68 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#15), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] Condition : isnotnull(cr_item_sk#13) -(20) ColumnarToRow [codegen id : 10] +(18) ColumnarToRow [codegen id : 10] Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] -(21) ReusedExchange [Reuses operator id: 7] +(19) ReusedExchange [Reuses operator id: 7] Output [2]: [i_item_sk#17, i_item_id#18] -(22) BroadcastHashJoin [codegen id : 10] +(20) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_item_sk#13] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(23) Project [codegen id : 10] +(21) Project [codegen id : 10] Output [3]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18] Input [5]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15, i_item_sk#17, i_item_id#18] -(24) ReusedExchange [Reuses operator id: 68] +(22) ReusedExchange [Reuses operator id: 62] Output [1]: [d_date_sk#19] -(25) BroadcastHashJoin [codegen id : 10] +(23) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_returned_date_sk#15] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(26) Project [codegen id : 10] +(24) Project [codegen id : 10] Output [2]: [cr_return_quantity#14, i_item_id#18] Input [4]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18, d_date_sk#19] -(27) HashAggregate [codegen id : 10] +(25) HashAggregate [codegen id : 10] Input [2]: [cr_return_quantity#14, i_item_id#18] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(cr_return_quantity#14)] Aggregate Attributes [1]: [sum#20] Results [2]: [i_item_id#18, sum#21] -(28) RowToColumnar -Input [2]: [i_item_id#18, sum#21] - -(29) CometColumnarExchange -Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) ColumnarToRow [codegen id : 11] +(26) Exchange Input [2]: [i_item_id#18, sum#21] +Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(31) HashAggregate [codegen id : 11] +(27) HashAggregate [codegen id : 11] Input [2]: [i_item_id#18, sum#21] Keys [1]: [i_item_id#18] Functions [1]: [sum(cr_return_quantity#14)] Aggregate Attributes [1]: [sum(cr_return_quantity#14)#22] Results [2]: [i_item_id#18 AS item_id#23, sum(cr_return_quantity#14)#22 AS cr_item_qty#24] -(32) BroadcastExchange +(28) BroadcastExchange Input [2]: [item_id#23, cr_item_qty#24] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 18] +(29) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#11] Right keys [1]: [item_id#23] Join type: Inner Join condition: None -(34) Project [codegen id : 18] +(30) Project [codegen id : 18] Output [3]: [item_id#11, sr_item_qty#12, cr_item_qty#24] Input [4]: [item_id#11, sr_item_qty#12, item_id#23, cr_item_qty#24] @@ -220,100 +202,94 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(36) CometFilter +(32) CometFilter Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] Condition : isnotnull(wr_item_sk#25) -(37) ColumnarToRow [codegen id : 16] +(33) ColumnarToRow [codegen id : 16] Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -(38) ReusedExchange [Reuses operator id: 7] +(34) ReusedExchange [Reuses operator id: 7] Output [2]: [i_item_sk#29, i_item_id#30] -(39) BroadcastHashJoin [codegen id : 16] +(35) BroadcastHashJoin [codegen id : 16] Left keys [1]: [wr_item_sk#25] Right keys [1]: [i_item_sk#29] Join type: Inner Join condition: None -(40) Project [codegen id : 16] +(36) Project [codegen id : 16] Output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#29, i_item_id#30] -(41) ReusedExchange [Reuses operator id: 68] +(37) ReusedExchange [Reuses operator id: 62] Output [1]: [d_date_sk#31] -(42) BroadcastHashJoin [codegen id : 16] +(38) BroadcastHashJoin [codegen id : 16] Left keys [1]: [wr_returned_date_sk#27] Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(43) Project [codegen id : 16] +(39) Project [codegen id : 16] Output [2]: [wr_return_quantity#26, i_item_id#30] Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30, d_date_sk#31] -(44) HashAggregate [codegen id : 16] +(40) HashAggregate [codegen id : 16] Input [2]: [wr_return_quantity#26, i_item_id#30] Keys [1]: [i_item_id#30] Functions [1]: [partial_sum(wr_return_quantity#26)] Aggregate Attributes [1]: [sum#32] Results [2]: [i_item_id#30, sum#33] -(45) RowToColumnar -Input [2]: [i_item_id#30, sum#33] - -(46) CometColumnarExchange +(41) Exchange Input [2]: [i_item_id#30, sum#33] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(47) ColumnarToRow [codegen id : 17] -Input [2]: [i_item_id#30, sum#33] - -(48) HashAggregate [codegen id : 17] +(42) HashAggregate [codegen id : 17] Input [2]: [i_item_id#30, sum#33] Keys [1]: [i_item_id#30] Functions [1]: [sum(wr_return_quantity#26)] Aggregate Attributes [1]: [sum(wr_return_quantity#26)#34] Results [2]: [i_item_id#30 AS item_id#35, sum(wr_return_quantity#26)#34 AS wr_item_qty#36] -(49) BroadcastExchange +(43) BroadcastExchange Input [2]: [item_id#35, wr_item_qty#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(50) BroadcastHashJoin [codegen id : 18] +(44) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#11] Right keys [1]: [item_id#35] Join type: Inner Join condition: None -(51) Project [codegen id : 18] +(45) Project [codegen id : 18] Output [8]: [item_id#11, sr_item_qty#12, (((cast(sr_item_qty#12 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS sr_dev#37, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS cr_dev#38, wr_item_qty#36, (((cast(wr_item_qty#36 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS wr_dev#39, (cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as decimal(20,0)) / 3.0) AS average#40] Input [5]: [item_id#11, sr_item_qty#12, cr_item_qty#24, item_id#35, wr_item_qty#36] -(52) TakeOrderedAndProject +(46) TakeOrderedAndProject Input [8]: [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] Arguments: 100, [item_id#11 ASC NULLS FIRST, sr_item_qty#12 ASC NULLS FIRST], [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (68) -+- * Project (67) - +- * BroadcastHashJoin LeftSemi BuildRight (66) - :- * ColumnarToRow (55) - : +- CometFilter (54) - : +- CometScan parquet spark_catalog.default.date_dim (53) - +- BroadcastExchange (65) - +- * Project (64) - +- * BroadcastHashJoin LeftSemi BuildRight (63) - :- * ColumnarToRow (57) - : +- CometScan parquet spark_catalog.default.date_dim (56) - +- BroadcastExchange (62) - +- * ColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan parquet spark_catalog.default.date_dim (58) +BroadcastExchange (62) ++- * Project (61) + +- * BroadcastHashJoin LeftSemi BuildRight (60) + :- * ColumnarToRow (49) + : +- CometFilter (48) + : +- CometScan parquet spark_catalog.default.date_dim (47) + +- BroadcastExchange (59) + +- * Project (58) + +- * BroadcastHashJoin LeftSemi BuildRight (57) + :- * ColumnarToRow (51) + : +- CometScan parquet spark_catalog.default.date_dim (50) + +- BroadcastExchange (56) + +- * ColumnarToRow (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) (unknown) Scan parquet spark_catalog.default.date_dim @@ -323,11 +299,11 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(54) CometFilter +(48) CometFilter Input [2]: [d_date_sk#7, d_date#41] Condition : isnotnull(d_date_sk#7) -(55) ColumnarToRow [codegen id : 3] +(49) ColumnarToRow [codegen id : 3] Input [2]: [d_date_sk#7, d_date#41] (unknown) Scan parquet spark_catalog.default.date_dim @@ -336,7 +312,7 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(57) ColumnarToRow [codegen id : 2] +(51) ColumnarToRow [codegen id : 2] Input [2]: [d_date#42, d_week_seq#43] (unknown) Scan parquet spark_catalog.default.date_dim @@ -345,51 +321,51 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(59) CometFilter +(53) CometFilter Input [2]: [d_date#44, d_week_seq#45] Condition : cast(d_date#44 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(60) CometProject +(54) CometProject Input [2]: [d_date#44, d_week_seq#45] Arguments: [d_week_seq#45], [d_week_seq#45] -(61) ColumnarToRow [codegen id : 1] +(55) ColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#45] -(62) BroadcastExchange +(56) BroadcastExchange Input [1]: [d_week_seq#45] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(63) BroadcastHashJoin [codegen id : 2] +(57) BroadcastHashJoin [codegen id : 2] Left keys [1]: [d_week_seq#43] Right keys [1]: [d_week_seq#45] Join type: LeftSemi Join condition: None -(64) Project [codegen id : 2] +(58) Project [codegen id : 2] Output [1]: [d_date#42] Input [2]: [d_date#42, d_week_seq#43] -(65) BroadcastExchange +(59) BroadcastExchange Input [1]: [d_date#42] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=8] -(66) BroadcastHashJoin [codegen id : 3] +(60) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#41] Right keys [1]: [d_date#42] Join type: LeftSemi Join condition: None -(67) Project [codegen id : 3] +(61) Project [codegen id : 3] Output [1]: [d_date_sk#7] Input [2]: [d_date_sk#7, d_date#41] -(68) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 18 Hosting Expression = cr_returned_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 16 Hosting Expression = cr_returned_date_sk#15 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 35 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 31 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index 1f02d66fef..a8f1ba3f10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -5,97 +5,91 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty Project [item_id,sr_item_qty,cr_item_qty] BroadcastHashJoin [item_id,item_id] HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - Project [sr_return_quantity,i_item_id] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [sr_return_quantity,sr_returned_date_sk,i_item_id] - BroadcastHashJoin [sr_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (3) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - ColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,sr_return_quantity] [sum,sum] + Project [sr_return_quantity,i_item_id] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [sr_return_quantity,sr_returned_date_sk,i_item_id] + BroadcastHashJoin [sr_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (3) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + ColumnarToRow + InputAdapter + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [d_date] + BroadcastHashJoin [d_week_seq,d_week_seq] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_date] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (11) HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [i_item_id,cr_return_quantity] [sum,sum] - Project [cr_return_quantity,i_item_id] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cr_return_quantity,cr_returned_date_sk,i_item_id] - BroadcastHashJoin [cr_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #9 - RowToColumnar - WholeStageCodegen (16) - HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - Project [wr_return_quantity,i_item_id] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Project [wr_return_quantity,wr_returned_date_sk,i_item_id] - BroadcastHashJoin [wr_item_sk,i_item_sk] + InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (10) + HashAggregate [i_item_id,cr_return_quantity] [sum,sum] + Project [cr_return_quantity,i_item_id] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Project [cr_return_quantity,cr_returned_date_sk,i_item_id] + BroadcastHashJoin [cr_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometFilter [wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + CometFilter [cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 InputAdapter ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (17) + HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] + InputAdapter + Exchange [i_item_id] #9 + WholeStageCodegen (16) + HashAggregate [i_item_id,wr_return_quantity] [sum,sum] + Project [wr_return_quantity,i_item_id] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Project [wr_return_quantity,wr_returned_date_sk,i_item_id] + BroadcastHashJoin [wr_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + InputAdapter + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index 2afc9c6c58..53329906e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -1,53 +1,51 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * HashAggregate (48) - +- * ColumnarToRow (47) - +- CometColumnarExchange (46) - +- RowToColumnar (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Project (27) - : : : +- * BroadcastHashJoin Inner BuildRight (26) - : : : :- * Project (21) - : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometBroadcastExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.web_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.web_page (10) - : : : : +- BroadcastExchange (19) - : : : : +- * ColumnarToRow (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (16) - : : : +- BroadcastExchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : +- BroadcastExchange (32) - : : +- * ColumnarToRow (31) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometScan parquet spark_catalog.default.customer_address (28) - : +- ReusedExchange (35) - +- BroadcastExchange (41) - +- * ColumnarToRow (40) - +- CometFilter (39) - +- CometScan parquet spark_catalog.default.reason (38) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (34) + : : +- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Project (27) + : : : +- * BroadcastHashJoin Inner BuildRight (26) + : : : :- * Project (21) + : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometBroadcastExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.web_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.web_page (10) + : : : : +- BroadcastExchange (19) + : : : : +- * ColumnarToRow (18) + : : : : +- CometFilter (17) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (16) + : : : +- BroadcastExchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) + : : +- BroadcastExchange (32) + : : +- * ColumnarToRow (31) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.customer_address (28) + : +- ReusedExchange (35) + +- BroadcastExchange (41) + +- * ColumnarToRow (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.reason (38) (unknown) Scan parquet spark_catalog.default.web_sales @@ -209,7 +207,7 @@ Join condition: ((((ca_state#26 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) A Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#25, ca_state#26] -(35) ReusedExchange [Reuses operator id: 54] +(35) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#28] (36) BroadcastHashJoin [codegen id : 7] @@ -257,35 +255,29 @@ Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunde Aggregate Attributes [6]: [sum#31, count#32, sum#33, count#34, sum#35, count#36] Results [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] -(45) RowToColumnar +(45) Exchange Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] +Arguments: hashpartitioning(r_reason_desc#30, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(46) CometColumnarExchange -Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Arguments: hashpartitioning(r_reason_desc#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(47) ColumnarToRow [codegen id : 8] -Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] - -(48) HashAggregate [codegen id : 8] +(46) HashAggregate [codegen id : 8] Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] Keys [1]: [r_reason_desc#30] Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] Aggregate Attributes [3]: [avg(ws_quantity#4)#43, avg(UnscaledValue(wr_refunded_cash#16))#44, avg(UnscaledValue(wr_fee#15))#45] Results [4]: [substr(r_reason_desc#30, 1, 20) AS substr(r_reason_desc, 1, 20)#46, avg(ws_quantity#4)#43 AS avg(ws_quantity)#47, cast((avg(UnscaledValue(wr_refunded_cash#16))#44 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#48, cast((avg(UnscaledValue(wr_fee#15))#45 / 100.0) as decimal(11,6)) AS avg(wr_fee)#49] -(49) TakeOrderedAndProject +(47) TakeOrderedAndProject Input [4]: [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] Arguments: 100, [substr(r_reason_desc, 1, 20)#46 ASC NULLS FIRST, avg(ws_quantity)#47 ASC NULLS FIRST, avg(wr_refunded_cash)#48 ASC NULLS FIRST, avg(wr_fee)#49 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim @@ -295,18 +287,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter +(49) CometFilter Input [2]: [d_date_sk#28, d_year#50] Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2000)) AND isnotnull(d_date_sk#28)) -(52) CometProject +(50) CometProject Input [2]: [d_date_sk#28, d_year#50] Arguments: [d_date_sk#28], [d_date_sk#28] -(53) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] -(54) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt index 91d0ae8d09..7c5ee727b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt @@ -1,77 +1,75 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] WholeStageCodegen (8) HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [r_reason_desc] #1 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - BroadcastHashJoin [wr_reason_sk,r_reason_sk] - Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] - Project [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - ColumnarToRow - InputAdapter - CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - CometBroadcastExchange #2 - CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + Exchange [r_reason_desc] #1 + WholeStageCodegen (7) + HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] + BroadcastHashJoin [wr_reason_sk,r_reason_sk] + Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] + BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + Project [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + ColumnarToRow + InputAdapter + CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + CometBroadcastExchange #2 + CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) + BroadcastExchange #6 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #7 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [r_reason_sk] + CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt index 8e40633a9e..cab784da58 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt @@ -1,29 +1,25 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * Project (24) - +- Window (23) - +- * ColumnarToRow (22) - +- CometSort (21) - +- CometColumnarExchange (20) - +- RowToColumnar (19) - +- * HashAggregate (18) - +- * ColumnarToRow (17) - +- CometColumnarExchange (16) - +- RowToColumnar (15) - +- * HashAggregate (14) - +- * Expand (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (10) - +- * ColumnarToRow (9) - +- CometFilter (8) - +- CometScan parquet spark_catalog.default.item (7) +TakeOrderedAndProject (21) ++- * Project (20) + +- Window (19) + +- * Sort (18) + +- Exchange (17) + +- * HashAggregate (16) + +- Exchange (15) + +- * HashAggregate (14) + +- * Expand (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- ReusedExchange (4) + +- BroadcastExchange (10) + +- * ColumnarToRow (9) + +- CometFilter (8) + +- CometScan parquet spark_catalog.default.item (7) (unknown) Scan parquet spark_catalog.default.web_sales @@ -41,7 +37,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 30] +(4) ReusedExchange [Reuses operator id: 26] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 3] @@ -93,57 +89,45 @@ Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum#12] Results [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] -(15) RowToColumnar +(15) Exchange Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] +Arguments: hashpartitioning(i_category#9, i_class#10, spark_grouping_id#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(16) CometColumnarExchange -Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] -Arguments: hashpartitioning(i_category#9, i_class#10, spark_grouping_id#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) ColumnarToRow [codegen id : 4] -Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] - -(18) HashAggregate [codegen id : 4] +(16) HashAggregate [codegen id : 4] Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] Keys [3]: [i_category#9, i_class#10, spark_grouping_id#11] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#14] Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS total_sum#15, i_category#9, i_class#10, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS lochierarchy#16, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS _w0#17, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS _w1#18, CASE WHEN (cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint) = 0) THEN i_category#9 END AS _w2#19] -(19) RowToColumnar -Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] - -(20) CometColumnarExchange -Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] -Arguments: hashpartitioning(_w1#18, _w2#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) CometSort +(17) Exchange Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] -Arguments: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19], [_w1#18 ASC NULLS FIRST, _w2#19 ASC NULLS FIRST, _w0#17 DESC NULLS LAST] +Arguments: hashpartitioning(_w1#18, _w2#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) ColumnarToRow [codegen id : 5] +(18) Sort [codegen id : 5] Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] +Arguments: [_w1#18 ASC NULLS FIRST, _w2#19 ASC NULLS FIRST, _w0#17 DESC NULLS LAST], false, 0 -(23) Window +(19) Window Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] Arguments: [rank(_w0#17) windowspecdefinition(_w1#18, _w2#19, _w0#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#20], [_w1#18, _w2#19], [_w0#17 DESC NULLS LAST] -(24) Project [codegen id : 6] +(20) Project [codegen id : 6] Output [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] Input [8]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19, rank_within_parent#20] -(25) TakeOrderedAndProject +(21) TakeOrderedAndProject Input [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] Arguments: 100, [lochierarchy#16 DESC NULLS LAST, CASE WHEN (lochierarchy#16 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#20 ASC NULLS FIRST], [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * ColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.date_dim (26) +BroadcastExchange (26) ++- * ColumnarToRow (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) (unknown) Scan parquet spark_catalog.default.date_dim @@ -153,18 +137,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter +(23) CometFilter Input [2]: [d_date_sk#5, d_month_seq#21] Condition : (((isnotnull(d_month_seq#21) AND (d_month_seq#21 >= 1200)) AND (d_month_seq#21 <= 1211)) AND isnotnull(d_date_sk#5)) -(28) CometProject +(24) CometProject Input [2]: [d_date_sk#5, d_month_seq#21] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) ColumnarToRow [codegen id : 1] +(25) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(30) BroadcastExchange +(26) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index 502cd9c52c..f9db2ce7a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -4,42 +4,38 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (5) - ColumnarToRow + Sort [_w1,_w2,_w0] InputAdapter - CometSort [_w1,_w2,_w0] - CometColumnarExchange [_w1,_w2] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [ws_net_paid,i_category,i_class] - Project [ws_net_paid,i_category,i_class] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + Exchange [_w1,_w2] #1 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + InputAdapter + Exchange [i_category,i_class,spark_grouping_id] #2 + WholeStageCodegen (3) + HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] + Expand [ws_net_paid,i_category,i_class] + Project [ws_net_paid,i_category,i_class] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index 14c482a3c8..a82b90a2a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -1,59 +1,51 @@ == Physical Plan == -* HashAggregate (55) -+- * ColumnarToRow (54) - +- CometColumnarExchange (53) - +- RowToColumnar (52) - +- * HashAggregate (51) - +- * Project (50) - +- * BroadcastHashJoin LeftAnti BuildRight (49) - :- * BroadcastHashJoin LeftAnti BuildRight (33) - : :- * ColumnarToRow (17) - : : +- CometHashAggregate (16) - : : +- CometColumnarExchange (15) - : : +- RowToColumnar (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer (7) - : +- BroadcastExchange (32) - : +- * ColumnarToRow (31) - : +- CometHashAggregate (30) - : +- CometColumnarExchange (29) - : +- RowToColumnar (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * ColumnarToRow (20) - : : : +- CometFilter (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (48) - +- * ColumnarToRow (47) - +- CometHashAggregate (46) - +- CometColumnarExchange (45) - +- RowToColumnar (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * ColumnarToRow (36) - : : +- CometFilter (35) - : : +- CometScan parquet spark_catalog.default.web_sales (34) - : +- ReusedExchange (37) - +- ReusedExchange (40) +* HashAggregate (47) ++- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin LeftAnti BuildRight (43) + :- * BroadcastHashJoin LeftAnti BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer (7) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- BroadcastExchange (42) + +- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * ColumnarToRow (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.web_sales (30) + : +- ReusedExchange (33) + +- ReusedExchange (36) (unknown) Scan parquet spark_catalog.default.store_sales @@ -71,7 +63,7 @@ Condition : isnotnull(ss_customer_sk#1) (3) ColumnarToRow [codegen id : 3] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -(4) ReusedExchange [Reuses operator id: 60] +(4) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#4, d_date#5] (5) BroadcastHashJoin [codegen id : 3] @@ -119,20 +111,16 @@ Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#8, c_first_name#7, d_date#5] -(14) RowToColumnar +(14) Exchange Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometHashAggregate +(15) HashAggregate [codegen id : 12] Input [3]: [c_last_name#8, c_first_name#7, d_date#5] Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] Functions: [] - -(17) ColumnarToRow [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Aggregate Attributes: [] +Results [3]: [c_last_name#8, c_first_name#7, d_date#5] (unknown) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] @@ -142,66 +130,62 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] Condition : isnotnull(cs_bill_customer_sk#9) -(20) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 6] Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] -(21) ReusedExchange [Reuses operator id: 60] +(19) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#12, d_date#13] -(22) BroadcastHashJoin [codegen id : 6] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(23) Project [codegen id : 6] +(21) Project [codegen id : 6] Output [2]: [cs_bill_customer_sk#9, d_date#13] Input [4]: [cs_bill_customer_sk#9, cs_sold_date_sk#10, d_date_sk#12, d_date#13] -(24) ReusedExchange [Reuses operator id: 10] +(22) ReusedExchange [Reuses operator id: 10] Output [3]: [c_customer_sk#14, c_first_name#15, c_last_name#16] -(25) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_bill_customer_sk#9] Right keys [1]: [c_customer_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [3]: [c_last_name#16, c_first_name#15, d_date#13] Input [5]: [cs_bill_customer_sk#9, d_date#13, c_customer_sk#14, c_first_name#15, c_last_name#16] -(27) HashAggregate [codegen id : 6] +(25) HashAggregate [codegen id : 6] Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(28) RowToColumnar +(26) Exchange Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(29) CometColumnarExchange -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) CometHashAggregate +(27) HashAggregate [codegen id : 7] Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(31) ColumnarToRow [codegen id : 7] -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] - -(32) BroadcastExchange +(28) BroadcastExchange Input [3]: [c_last_name#16, c_first_name#15, d_date#13] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 12] +(29) BroadcastHashJoin [codegen id : 12] Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#16, ), isnull(c_last_name#16), coalesce(c_first_name#15, ), isnull(c_first_name#15), coalesce(d_date#13, 1970-01-01), isnull(d_date#13)] Join type: LeftAnti @@ -215,93 +199,83 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#18), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] Condition : isnotnull(ws_bill_customer_sk#17) -(36) ColumnarToRow [codegen id : 10] +(32) ColumnarToRow [codegen id : 10] Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] -(37) ReusedExchange [Reuses operator id: 60] +(33) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#20, d_date#21] -(38) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#18] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(39) Project [codegen id : 10] +(35) Project [codegen id : 10] Output [2]: [ws_bill_customer_sk#17, d_date#21] Input [4]: [ws_bill_customer_sk#17, ws_sold_date_sk#18, d_date_sk#20, d_date#21] -(40) ReusedExchange [Reuses operator id: 10] +(36) ReusedExchange [Reuses operator id: 10] Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] -(41) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_bill_customer_sk#17] Right keys [1]: [c_customer_sk#22] Join type: Inner Join condition: None -(42) Project [codegen id : 10] +(38) Project [codegen id : 10] Output [3]: [c_last_name#24, c_first_name#23, d_date#21] Input [5]: [ws_bill_customer_sk#17, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] -(43) HashAggregate [codegen id : 10] +(39) HashAggregate [codegen id : 10] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(44) RowToColumnar -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(45) CometColumnarExchange +(40) Exchange Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(46) CometHashAggregate +(41) HashAggregate [codegen id : 11] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(47) ColumnarToRow [codegen id : 11] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(48) BroadcastExchange +(42) BroadcastExchange Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 12] +(43) BroadcastHashJoin [codegen id : 12] Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] Join type: LeftAnti Join condition: None -(50) Project [codegen id : 12] +(44) Project [codegen id : 12] Output: [] Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -(51) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#25] Results [1]: [count#26] -(52) RowToColumnar -Input [1]: [count#26] - -(53) CometColumnarExchange -Input [1]: [count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(54) ColumnarToRow [codegen id : 13] +(46) Exchange Input [1]: [count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(55) HashAggregate [codegen id : 13] +(47) HashAggregate [codegen id : 13] Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] @@ -311,11 +285,11 @@ Results [1]: [count(1)#27 AS count(1)#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (60) -+- * ColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.date_dim (56) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) (unknown) Scan parquet spark_catalog.default.date_dim @@ -325,23 +299,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter +(49) CometFilter Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) -(58) CometProject +(50) CometProject Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(59) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(60) BroadcastExchange +(52) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 +Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 34 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index c3e01e4f26..315afe6602 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -1,89 +1,81 @@ WholeStageCodegen (13) HashAggregate [count] [count(1),count(1),count] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (12) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - ColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + InputAdapter + Exchange #1 + WholeStageCodegen (12) + HashAggregate [count,count] + Project + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index 73c5b73f6e..26821dfd2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt @@ -1,202 +1,186 @@ == Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (198) -:- * BroadcastNestedLoopJoin Inner BuildRight (174) -: :- * BroadcastNestedLoopJoin Inner BuildRight (150) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (126) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (102) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (78) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (54) -: : : : : : :- * HashAggregate (30) -: : : : : : : +- * ColumnarToRow (29) -: : : : : : : +- CometColumnarExchange (28) -: : : : : : : +- RowToColumnar (27) -: : : : : : : +- * HashAggregate (26) -: : : : : : : +- * Project (25) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (24) -: : : : : : : :- * Project (18) -: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) -: : : : : : : : :- * Project (11) -: : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (10) -: : : : : : : : : :- * ColumnarToRow (4) -: : : : : : : : : : +- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- BroadcastExchange (9) -: : : : : : : : : +- * ColumnarToRow (8) -: : : : : : : : : +- CometProject (7) -: : : : : : : : : +- CometFilter (6) -: : : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (5) -: : : : : : : : +- BroadcastExchange (16) -: : : : : : : : +- * ColumnarToRow (15) -: : : : : : : : +- CometProject (14) -: : : : : : : : +- CometFilter (13) -: : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (12) -: : : : : : : +- BroadcastExchange (23) -: : : : : : : +- * ColumnarToRow (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometFilter (20) -: : : : : : : +- CometScan parquet spark_catalog.default.store (19) -: : : : : : +- BroadcastExchange (53) -: : : : : : +- * HashAggregate (52) -: : : : : : +- * ColumnarToRow (51) -: : : : : : +- CometColumnarExchange (50) -: : : : : : +- RowToColumnar (49) -: : : : : : +- * HashAggregate (48) -: : : : : : +- * Project (47) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (46) -: : : : : : :- * Project (44) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (43) -: : : : : : : :- * Project (37) -: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (36) -: : : : : : : : :- * ColumnarToRow (34) -: : : : : : : : : +- CometProject (33) -: : : : : : : : : +- CometFilter (32) -: : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (31) -: : : : : : : : +- ReusedExchange (35) -: : : : : : : +- BroadcastExchange (42) -: : : : : : : +- * ColumnarToRow (41) -: : : : : : : +- CometProject (40) -: : : : : : : +- CometFilter (39) -: : : : : : : +- CometScan parquet spark_catalog.default.time_dim (38) -: : : : : : +- ReusedExchange (45) -: : : : : +- BroadcastExchange (77) -: : : : : +- * HashAggregate (76) -: : : : : +- * ColumnarToRow (75) -: : : : : +- CometColumnarExchange (74) -: : : : : +- RowToColumnar (73) -: : : : : +- * HashAggregate (72) -: : : : : +- * Project (71) -: : : : : +- * BroadcastHashJoin Inner BuildRight (70) -: : : : : :- * Project (68) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (67) -: : : : : : :- * Project (61) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) -: : : : : : : :- * ColumnarToRow (58) -: : : : : : : : +- CometProject (57) -: : : : : : : : +- CometFilter (56) -: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (55) -: : : : : : : +- ReusedExchange (59) -: : : : : : +- BroadcastExchange (66) -: : : : : : +- * ColumnarToRow (65) -: : : : : : +- CometProject (64) -: : : : : : +- CometFilter (63) -: : : : : : +- CometScan parquet spark_catalog.default.time_dim (62) -: : : : : +- ReusedExchange (69) -: : : : +- BroadcastExchange (101) -: : : : +- * HashAggregate (100) -: : : : +- * ColumnarToRow (99) -: : : : +- CometColumnarExchange (98) -: : : : +- RowToColumnar (97) -: : : : +- * HashAggregate (96) -: : : : +- * Project (95) -: : : : +- * BroadcastHashJoin Inner BuildRight (94) -: : : : :- * Project (92) -: : : : : +- * BroadcastHashJoin Inner BuildRight (91) -: : : : : :- * Project (85) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (84) -: : : : : : :- * ColumnarToRow (82) -: : : : : : : +- CometProject (81) -: : : : : : : +- CometFilter (80) -: : : : : : : +- CometScan parquet spark_catalog.default.store_sales (79) -: : : : : : +- ReusedExchange (83) -: : : : : +- BroadcastExchange (90) -: : : : : +- * ColumnarToRow (89) -: : : : : +- CometProject (88) -: : : : : +- CometFilter (87) -: : : : : +- CometScan parquet spark_catalog.default.time_dim (86) -: : : : +- ReusedExchange (93) -: : : +- BroadcastExchange (125) -: : : +- * HashAggregate (124) -: : : +- * ColumnarToRow (123) -: : : +- CometColumnarExchange (122) -: : : +- RowToColumnar (121) -: : : +- * HashAggregate (120) -: : : +- * Project (119) -: : : +- * BroadcastHashJoin Inner BuildRight (118) -: : : :- * Project (116) -: : : : +- * BroadcastHashJoin Inner BuildRight (115) -: : : : :- * Project (109) -: : : : : +- * BroadcastHashJoin Inner BuildRight (108) -: : : : : :- * ColumnarToRow (106) -: : : : : : +- CometProject (105) -: : : : : : +- CometFilter (104) -: : : : : : +- CometScan parquet spark_catalog.default.store_sales (103) -: : : : : +- ReusedExchange (107) -: : : : +- BroadcastExchange (114) -: : : : +- * ColumnarToRow (113) -: : : : +- CometProject (112) -: : : : +- CometFilter (111) -: : : : +- CometScan parquet spark_catalog.default.time_dim (110) -: : : +- ReusedExchange (117) -: : +- BroadcastExchange (149) -: : +- * HashAggregate (148) -: : +- * ColumnarToRow (147) -: : +- CometColumnarExchange (146) -: : +- RowToColumnar (145) -: : +- * HashAggregate (144) -: : +- * Project (143) -: : +- * BroadcastHashJoin Inner BuildRight (142) -: : :- * Project (140) -: : : +- * BroadcastHashJoin Inner BuildRight (139) -: : : :- * Project (133) -: : : : +- * BroadcastHashJoin Inner BuildRight (132) -: : : : :- * ColumnarToRow (130) -: : : : : +- CometProject (129) -: : : : : +- CometFilter (128) -: : : : : +- CometScan parquet spark_catalog.default.store_sales (127) -: : : : +- ReusedExchange (131) -: : : +- BroadcastExchange (138) -: : : +- * ColumnarToRow (137) -: : : +- CometProject (136) -: : : +- CometFilter (135) -: : : +- CometScan parquet spark_catalog.default.time_dim (134) -: : +- ReusedExchange (141) -: +- BroadcastExchange (173) -: +- * HashAggregate (172) -: +- * ColumnarToRow (171) -: +- CometColumnarExchange (170) -: +- RowToColumnar (169) -: +- * HashAggregate (168) -: +- * Project (167) -: +- * BroadcastHashJoin Inner BuildRight (166) -: :- * Project (164) -: : +- * BroadcastHashJoin Inner BuildRight (163) -: : :- * Project (157) -: : : +- * BroadcastHashJoin Inner BuildRight (156) -: : : :- * ColumnarToRow (154) -: : : : +- CometProject (153) -: : : : +- CometFilter (152) -: : : : +- CometScan parquet spark_catalog.default.store_sales (151) -: : : +- ReusedExchange (155) -: : +- BroadcastExchange (162) -: : +- * ColumnarToRow (161) -: : +- CometProject (160) -: : +- CometFilter (159) -: : +- CometScan parquet spark_catalog.default.time_dim (158) -: +- ReusedExchange (165) -+- BroadcastExchange (197) - +- * HashAggregate (196) - +- * ColumnarToRow (195) - +- CometColumnarExchange (194) - +- RowToColumnar (193) - +- * HashAggregate (192) - +- * Project (191) - +- * BroadcastHashJoin Inner BuildRight (190) - :- * Project (188) - : +- * BroadcastHashJoin Inner BuildRight (187) - : :- * Project (181) - : : +- * BroadcastHashJoin Inner BuildRight (180) - : : :- * ColumnarToRow (178) - : : : +- CometProject (177) - : : : +- CometFilter (176) - : : : +- CometScan parquet spark_catalog.default.store_sales (175) - : : +- ReusedExchange (179) - : +- BroadcastExchange (186) - : +- * ColumnarToRow (185) - : +- CometProject (184) - : +- CometFilter (183) - : +- CometScan parquet spark_catalog.default.time_dim (182) - +- ReusedExchange (189) +* BroadcastNestedLoopJoin Inner BuildRight (182) +:- * BroadcastNestedLoopJoin Inner BuildRight (160) +: :- * BroadcastNestedLoopJoin Inner BuildRight (138) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (116) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (94) +: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (72) +: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (50) +: : : : : : :- * HashAggregate (28) +: : : : : : : +- Exchange (27) +: : : : : : : +- * HashAggregate (26) +: : : : : : : +- * Project (25) +: : : : : : : +- * BroadcastHashJoin Inner BuildRight (24) +: : : : : : : :- * Project (18) +: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) +: : : : : : : : :- * Project (11) +: : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (10) +: : : : : : : : : :- * ColumnarToRow (4) +: : : : : : : : : : +- CometProject (3) +: : : : : : : : : : +- CometFilter (2) +: : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : : : : : : +- BroadcastExchange (9) +: : : : : : : : : +- * ColumnarToRow (8) +: : : : : : : : : +- CometProject (7) +: : : : : : : : : +- CometFilter (6) +: : : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (5) +: : : : : : : : +- BroadcastExchange (16) +: : : : : : : : +- * ColumnarToRow (15) +: : : : : : : : +- CometProject (14) +: : : : : : : : +- CometFilter (13) +: : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (12) +: : : : : : : +- BroadcastExchange (23) +: : : : : : : +- * ColumnarToRow (22) +: : : : : : : +- CometProject (21) +: : : : : : : +- CometFilter (20) +: : : : : : : +- CometScan parquet spark_catalog.default.store (19) +: : : : : : +- BroadcastExchange (49) +: : : : : : +- * HashAggregate (48) +: : : : : : +- Exchange (47) +: : : : : : +- * HashAggregate (46) +: : : : : : +- * Project (45) +: : : : : : +- * BroadcastHashJoin Inner BuildRight (44) +: : : : : : :- * Project (42) +: : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) +: : : : : : : :- * Project (35) +: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) +: : : : : : : : :- * ColumnarToRow (32) +: : : : : : : : : +- CometProject (31) +: : : : : : : : : +- CometFilter (30) +: : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) +: : : : : : : : +- ReusedExchange (33) +: : : : : : : +- BroadcastExchange (40) +: : : : : : : +- * ColumnarToRow (39) +: : : : : : : +- CometProject (38) +: : : : : : : +- CometFilter (37) +: : : : : : : +- CometScan parquet spark_catalog.default.time_dim (36) +: : : : : : +- ReusedExchange (43) +: : : : : +- BroadcastExchange (71) +: : : : : +- * HashAggregate (70) +: : : : : +- Exchange (69) +: : : : : +- * HashAggregate (68) +: : : : : +- * Project (67) +: : : : : +- * BroadcastHashJoin Inner BuildRight (66) +: : : : : :- * Project (64) +: : : : : : +- * BroadcastHashJoin Inner BuildRight (63) +: : : : : : :- * Project (57) +: : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) +: : : : : : : :- * ColumnarToRow (54) +: : : : : : : : +- CometProject (53) +: : : : : : : : +- CometFilter (52) +: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (51) +: : : : : : : +- ReusedExchange (55) +: : : : : : +- BroadcastExchange (62) +: : : : : : +- * ColumnarToRow (61) +: : : : : : +- CometProject (60) +: : : : : : +- CometFilter (59) +: : : : : : +- CometScan parquet spark_catalog.default.time_dim (58) +: : : : : +- ReusedExchange (65) +: : : : +- BroadcastExchange (93) +: : : : +- * HashAggregate (92) +: : : : +- Exchange (91) +: : : : +- * HashAggregate (90) +: : : : +- * Project (89) +: : : : +- * BroadcastHashJoin Inner BuildRight (88) +: : : : :- * Project (86) +: : : : : +- * BroadcastHashJoin Inner BuildRight (85) +: : : : : :- * Project (79) +: : : : : : +- * BroadcastHashJoin Inner BuildRight (78) +: : : : : : :- * ColumnarToRow (76) +: : : : : : : +- CometProject (75) +: : : : : : : +- CometFilter (74) +: : : : : : : +- CometScan parquet spark_catalog.default.store_sales (73) +: : : : : : +- ReusedExchange (77) +: : : : : +- BroadcastExchange (84) +: : : : : +- * ColumnarToRow (83) +: : : : : +- CometProject (82) +: : : : : +- CometFilter (81) +: : : : : +- CometScan parquet spark_catalog.default.time_dim (80) +: : : : +- ReusedExchange (87) +: : : +- BroadcastExchange (115) +: : : +- * HashAggregate (114) +: : : +- Exchange (113) +: : : +- * HashAggregate (112) +: : : +- * Project (111) +: : : +- * BroadcastHashJoin Inner BuildRight (110) +: : : :- * Project (108) +: : : : +- * BroadcastHashJoin Inner BuildRight (107) +: : : : :- * Project (101) +: : : : : +- * BroadcastHashJoin Inner BuildRight (100) +: : : : : :- * ColumnarToRow (98) +: : : : : : +- CometProject (97) +: : : : : : +- CometFilter (96) +: : : : : : +- CometScan parquet spark_catalog.default.store_sales (95) +: : : : : +- ReusedExchange (99) +: : : : +- BroadcastExchange (106) +: : : : +- * ColumnarToRow (105) +: : : : +- CometProject (104) +: : : : +- CometFilter (103) +: : : : +- CometScan parquet spark_catalog.default.time_dim (102) +: : : +- ReusedExchange (109) +: : +- BroadcastExchange (137) +: : +- * HashAggregate (136) +: : +- Exchange (135) +: : +- * HashAggregate (134) +: : +- * Project (133) +: : +- * BroadcastHashJoin Inner BuildRight (132) +: : :- * Project (130) +: : : +- * BroadcastHashJoin Inner BuildRight (129) +: : : :- * Project (123) +: : : : +- * BroadcastHashJoin Inner BuildRight (122) +: : : : :- * ColumnarToRow (120) +: : : : : +- CometProject (119) +: : : : : +- CometFilter (118) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (117) +: : : : +- ReusedExchange (121) +: : : +- BroadcastExchange (128) +: : : +- * ColumnarToRow (127) +: : : +- CometProject (126) +: : : +- CometFilter (125) +: : : +- CometScan parquet spark_catalog.default.time_dim (124) +: : +- ReusedExchange (131) +: +- BroadcastExchange (159) +: +- * HashAggregate (158) +: +- Exchange (157) +: +- * HashAggregate (156) +: +- * Project (155) +: +- * BroadcastHashJoin Inner BuildRight (154) +: :- * Project (152) +: : +- * BroadcastHashJoin Inner BuildRight (151) +: : :- * Project (145) +: : : +- * BroadcastHashJoin Inner BuildRight (144) +: : : :- * ColumnarToRow (142) +: : : : +- CometProject (141) +: : : : +- CometFilter (140) +: : : : +- CometScan parquet spark_catalog.default.store_sales (139) +: : : +- ReusedExchange (143) +: : +- BroadcastExchange (150) +: : +- * ColumnarToRow (149) +: : +- CometProject (148) +: : +- CometFilter (147) +: : +- CometScan parquet spark_catalog.default.time_dim (146) +: +- ReusedExchange (153) ++- BroadcastExchange (181) + +- * HashAggregate (180) + +- Exchange (179) + +- * HashAggregate (178) + +- * Project (177) + +- * BroadcastHashJoin Inner BuildRight (176) + :- * Project (174) + : +- * BroadcastHashJoin Inner BuildRight (173) + : :- * Project (167) + : : +- * BroadcastHashJoin Inner BuildRight (166) + : : :- * ColumnarToRow (164) + : : : +- CometProject (163) + : : : +- CometFilter (162) + : : : +- CometScan parquet spark_catalog.default.store_sales (161) + : : +- ReusedExchange (165) + : +- BroadcastExchange (172) + : +- * ColumnarToRow (171) + : +- CometProject (170) + : +- CometFilter (169) + : +- CometScan parquet spark_catalog.default.time_dim (168) + +- ReusedExchange (175) (unknown) Scan parquet spark_catalog.default.store_sales @@ -320,17 +304,11 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#13] Results [1]: [count#14] -(27) RowToColumnar +(27) Exchange Input [1]: [count#14] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(28) CometColumnarExchange -Input [1]: [count#14] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) ColumnarToRow [codegen id : 40] -Input [1]: [count#14] - -(30) HashAggregate [codegen id : 40] +(28) HashAggregate [codegen id : 40] Input [1]: [count#14] Keys: [] Functions [1]: [count(1)] @@ -344,27 +322,27 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(32) CometFilter +(30) CometFilter Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] Condition : ((isnotnull(ss_hdemo_sk#18) AND isnotnull(ss_sold_time_sk#17)) AND isnotnull(ss_store_sk#19)) -(33) CometProject +(31) CometProject Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] Arguments: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19], [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] -(34) ColumnarToRow [codegen id : 8] +(32) ColumnarToRow [codegen id : 8] Input [3]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] -(35) ReusedExchange [Reuses operator id: 9] +(33) ReusedExchange [Reuses operator id: 9] Output [1]: [hd_demo_sk#21] -(36) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_hdemo_sk#18] Right keys [1]: [hd_demo_sk#21] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(35) Project [codegen id : 8] Output [2]: [ss_sold_time_sk#17, ss_store_sk#19] Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, hd_demo_sk#21] @@ -375,73 +353,67 @@ Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(39) CometFilter +(37) CometFilter Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] Condition : ((((isnotnull(t_hour#23) AND isnotnull(t_minute#24)) AND (t_hour#23 = 9)) AND (t_minute#24 < 30)) AND isnotnull(t_time_sk#22)) -(40) CometProject +(38) CometProject Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] Arguments: [t_time_sk#22], [t_time_sk#22] -(41) ColumnarToRow [codegen id : 6] +(39) ColumnarToRow [codegen id : 6] Input [1]: [t_time_sk#22] -(42) BroadcastExchange +(40) BroadcastExchange Input [1]: [t_time_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 8] +(41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_sold_time_sk#17] Right keys [1]: [t_time_sk#22] Join type: Inner Join condition: None -(44) Project [codegen id : 8] +(42) Project [codegen id : 8] Output [1]: [ss_store_sk#19] Input [3]: [ss_sold_time_sk#17, ss_store_sk#19, t_time_sk#22] -(45) ReusedExchange [Reuses operator id: 23] +(43) ReusedExchange [Reuses operator id: 23] Output [1]: [s_store_sk#25] -(46) BroadcastHashJoin [codegen id : 8] +(44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#19] Right keys [1]: [s_store_sk#25] Join type: Inner Join condition: None -(47) Project [codegen id : 8] +(45) Project [codegen id : 8] Output: [] Input [2]: [ss_store_sk#19, s_store_sk#25] -(48) HashAggregate [codegen id : 8] +(46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#26] Results [1]: [count#27] -(49) RowToColumnar +(47) Exchange Input [1]: [count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(50) CometColumnarExchange -Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(51) ColumnarToRow [codegen id : 9] -Input [1]: [count#27] - -(52) HashAggregate [codegen id : 9] +(48) HashAggregate [codegen id : 9] Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#28] Results [1]: [count(1)#28 AS h9_to_9_30#29] -(53) BroadcastExchange +(49) BroadcastExchange Input [1]: [h9_to_9_30#29] Arguments: IdentityBroadcastMode, [plan_id=7] -(54) BroadcastNestedLoopJoin [codegen id : 40] +(50) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None @@ -452,27 +424,27 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(56) CometFilter +(52) CometFilter Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] Condition : ((isnotnull(ss_hdemo_sk#31) AND isnotnull(ss_sold_time_sk#30)) AND isnotnull(ss_store_sk#32)) -(57) CometProject +(53) CometProject Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] Arguments: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32], [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] -(58) ColumnarToRow [codegen id : 13] +(54) ColumnarToRow [codegen id : 13] Input [3]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] -(59) ReusedExchange [Reuses operator id: 9] +(55) ReusedExchange [Reuses operator id: 9] Output [1]: [hd_demo_sk#34] -(60) BroadcastHashJoin [codegen id : 13] +(56) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_hdemo_sk#31] Right keys [1]: [hd_demo_sk#34] Join type: Inner Join condition: None -(61) Project [codegen id : 13] +(57) Project [codegen id : 13] Output [2]: [ss_sold_time_sk#30, ss_store_sk#32] Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, hd_demo_sk#34] @@ -483,73 +455,67 @@ Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(63) CometFilter +(59) CometFilter Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] Condition : ((((isnotnull(t_hour#36) AND isnotnull(t_minute#37)) AND (t_hour#36 = 9)) AND (t_minute#37 >= 30)) AND isnotnull(t_time_sk#35)) -(64) CometProject +(60) CometProject Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] Arguments: [t_time_sk#35], [t_time_sk#35] -(65) ColumnarToRow [codegen id : 11] +(61) ColumnarToRow [codegen id : 11] Input [1]: [t_time_sk#35] -(66) BroadcastExchange +(62) BroadcastExchange Input [1]: [t_time_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(67) BroadcastHashJoin [codegen id : 13] +(63) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_sold_time_sk#30] Right keys [1]: [t_time_sk#35] Join type: Inner Join condition: None -(68) Project [codegen id : 13] +(64) Project [codegen id : 13] Output [1]: [ss_store_sk#32] Input [3]: [ss_sold_time_sk#30, ss_store_sk#32, t_time_sk#35] -(69) ReusedExchange [Reuses operator id: 23] +(65) ReusedExchange [Reuses operator id: 23] Output [1]: [s_store_sk#38] -(70) BroadcastHashJoin [codegen id : 13] +(66) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_store_sk#32] Right keys [1]: [s_store_sk#38] Join type: Inner Join condition: None -(71) Project [codegen id : 13] +(67) Project [codegen id : 13] Output: [] Input [2]: [ss_store_sk#32, s_store_sk#38] -(72) HashAggregate [codegen id : 13] +(68) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#39] Results [1]: [count#40] -(73) RowToColumnar -Input [1]: [count#40] - -(74) CometColumnarExchange -Input [1]: [count#40] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(75) ColumnarToRow [codegen id : 14] +(69) Exchange Input [1]: [count#40] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(76) HashAggregate [codegen id : 14] +(70) HashAggregate [codegen id : 14] Input [1]: [count#40] Keys: [] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#41] Results [1]: [count(1)#41 AS h9_30_to_10#42] -(77) BroadcastExchange +(71) BroadcastExchange Input [1]: [h9_30_to_10#42] Arguments: IdentityBroadcastMode, [plan_id=10] -(78) BroadcastNestedLoopJoin [codegen id : 40] +(72) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None @@ -560,27 +526,27 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(80) CometFilter +(74) CometFilter Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] Condition : ((isnotnull(ss_hdemo_sk#44) AND isnotnull(ss_sold_time_sk#43)) AND isnotnull(ss_store_sk#45)) -(81) CometProject +(75) CometProject Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] Arguments: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45], [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] -(82) ColumnarToRow [codegen id : 18] +(76) ColumnarToRow [codegen id : 18] Input [3]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] -(83) ReusedExchange [Reuses operator id: 9] +(77) ReusedExchange [Reuses operator id: 9] Output [1]: [hd_demo_sk#47] -(84) BroadcastHashJoin [codegen id : 18] +(78) BroadcastHashJoin [codegen id : 18] Left keys [1]: [ss_hdemo_sk#44] Right keys [1]: [hd_demo_sk#47] Join type: Inner Join condition: None -(85) Project [codegen id : 18] +(79) Project [codegen id : 18] Output [2]: [ss_sold_time_sk#43, ss_store_sk#45] Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, hd_demo_sk#47] @@ -591,73 +557,67 @@ Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(87) CometFilter +(81) CometFilter Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] Condition : ((((isnotnull(t_hour#49) AND isnotnull(t_minute#50)) AND (t_hour#49 = 10)) AND (t_minute#50 < 30)) AND isnotnull(t_time_sk#48)) -(88) CometProject +(82) CometProject Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] Arguments: [t_time_sk#48], [t_time_sk#48] -(89) ColumnarToRow [codegen id : 16] +(83) ColumnarToRow [codegen id : 16] Input [1]: [t_time_sk#48] -(90) BroadcastExchange +(84) BroadcastExchange Input [1]: [t_time_sk#48] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(91) BroadcastHashJoin [codegen id : 18] +(85) BroadcastHashJoin [codegen id : 18] Left keys [1]: [ss_sold_time_sk#43] Right keys [1]: [t_time_sk#48] Join type: Inner Join condition: None -(92) Project [codegen id : 18] +(86) Project [codegen id : 18] Output [1]: [ss_store_sk#45] Input [3]: [ss_sold_time_sk#43, ss_store_sk#45, t_time_sk#48] -(93) ReusedExchange [Reuses operator id: 23] +(87) ReusedExchange [Reuses operator id: 23] Output [1]: [s_store_sk#51] -(94) BroadcastHashJoin [codegen id : 18] +(88) BroadcastHashJoin [codegen id : 18] Left keys [1]: [ss_store_sk#45] Right keys [1]: [s_store_sk#51] Join type: Inner Join condition: None -(95) Project [codegen id : 18] +(89) Project [codegen id : 18] Output: [] Input [2]: [ss_store_sk#45, s_store_sk#51] -(96) HashAggregate [codegen id : 18] +(90) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#52] Results [1]: [count#53] -(97) RowToColumnar -Input [1]: [count#53] - -(98) CometColumnarExchange +(91) Exchange Input [1]: [count#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(99) ColumnarToRow [codegen id : 19] -Input [1]: [count#53] - -(100) HashAggregate [codegen id : 19] +(92) HashAggregate [codegen id : 19] Input [1]: [count#53] Keys: [] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#54] Results [1]: [count(1)#54 AS h10_to_10_30#55] -(101) BroadcastExchange +(93) BroadcastExchange Input [1]: [h10_to_10_30#55] Arguments: IdentityBroadcastMode, [plan_id=13] -(102) BroadcastNestedLoopJoin [codegen id : 40] +(94) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None @@ -668,27 +628,27 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(104) CometFilter +(96) CometFilter Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] Condition : ((isnotnull(ss_hdemo_sk#57) AND isnotnull(ss_sold_time_sk#56)) AND isnotnull(ss_store_sk#58)) -(105) CometProject +(97) CometProject Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] Arguments: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58], [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] -(106) ColumnarToRow [codegen id : 23] +(98) ColumnarToRow [codegen id : 23] Input [3]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] -(107) ReusedExchange [Reuses operator id: 9] +(99) ReusedExchange [Reuses operator id: 9] Output [1]: [hd_demo_sk#60] -(108) BroadcastHashJoin [codegen id : 23] +(100) BroadcastHashJoin [codegen id : 23] Left keys [1]: [ss_hdemo_sk#57] Right keys [1]: [hd_demo_sk#60] Join type: Inner Join condition: None -(109) Project [codegen id : 23] +(101) Project [codegen id : 23] Output [2]: [ss_sold_time_sk#56, ss_store_sk#58] Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, hd_demo_sk#60] @@ -699,73 +659,67 @@ Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(111) CometFilter +(103) CometFilter Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] Condition : ((((isnotnull(t_hour#62) AND isnotnull(t_minute#63)) AND (t_hour#62 = 10)) AND (t_minute#63 >= 30)) AND isnotnull(t_time_sk#61)) -(112) CometProject +(104) CometProject Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] Arguments: [t_time_sk#61], [t_time_sk#61] -(113) ColumnarToRow [codegen id : 21] +(105) ColumnarToRow [codegen id : 21] Input [1]: [t_time_sk#61] -(114) BroadcastExchange +(106) BroadcastExchange Input [1]: [t_time_sk#61] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -(115) BroadcastHashJoin [codegen id : 23] +(107) BroadcastHashJoin [codegen id : 23] Left keys [1]: [ss_sold_time_sk#56] Right keys [1]: [t_time_sk#61] Join type: Inner Join condition: None -(116) Project [codegen id : 23] +(108) Project [codegen id : 23] Output [1]: [ss_store_sk#58] Input [3]: [ss_sold_time_sk#56, ss_store_sk#58, t_time_sk#61] -(117) ReusedExchange [Reuses operator id: 23] +(109) ReusedExchange [Reuses operator id: 23] Output [1]: [s_store_sk#64] -(118) BroadcastHashJoin [codegen id : 23] +(110) BroadcastHashJoin [codegen id : 23] Left keys [1]: [ss_store_sk#58] Right keys [1]: [s_store_sk#64] Join type: Inner Join condition: None -(119) Project [codegen id : 23] +(111) Project [codegen id : 23] Output: [] Input [2]: [ss_store_sk#58, s_store_sk#64] -(120) HashAggregate [codegen id : 23] +(112) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#65] Results [1]: [count#66] -(121) RowToColumnar -Input [1]: [count#66] - -(122) CometColumnarExchange +(113) Exchange Input [1]: [count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] -(123) ColumnarToRow [codegen id : 24] -Input [1]: [count#66] - -(124) HashAggregate [codegen id : 24] +(114) HashAggregate [codegen id : 24] Input [1]: [count#66] Keys: [] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#67] Results [1]: [count(1)#67 AS h10_30_to_11#68] -(125) BroadcastExchange +(115) BroadcastExchange Input [1]: [h10_30_to_11#68] Arguments: IdentityBroadcastMode, [plan_id=16] -(126) BroadcastNestedLoopJoin [codegen id : 40] +(116) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None @@ -776,27 +730,27 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(128) CometFilter +(118) CometFilter Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) -(129) CometProject +(119) CometProject Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] Arguments: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71], [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] -(130) ColumnarToRow [codegen id : 28] +(120) ColumnarToRow [codegen id : 28] Input [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] -(131) ReusedExchange [Reuses operator id: 9] +(121) ReusedExchange [Reuses operator id: 9] Output [1]: [hd_demo_sk#73] -(132) BroadcastHashJoin [codegen id : 28] +(122) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_hdemo_sk#70] Right keys [1]: [hd_demo_sk#73] Join type: Inner Join condition: None -(133) Project [codegen id : 28] +(123) Project [codegen id : 28] Output [2]: [ss_sold_time_sk#69, ss_store_sk#71] Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, hd_demo_sk#73] @@ -807,73 +761,67 @@ Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(135) CometFilter +(125) CometFilter Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] Condition : ((((isnotnull(t_hour#75) AND isnotnull(t_minute#76)) AND (t_hour#75 = 11)) AND (t_minute#76 < 30)) AND isnotnull(t_time_sk#74)) -(136) CometProject +(126) CometProject Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] Arguments: [t_time_sk#74], [t_time_sk#74] -(137) ColumnarToRow [codegen id : 26] +(127) ColumnarToRow [codegen id : 26] Input [1]: [t_time_sk#74] -(138) BroadcastExchange +(128) BroadcastExchange Input [1]: [t_time_sk#74] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] -(139) BroadcastHashJoin [codegen id : 28] +(129) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_sold_time_sk#69] Right keys [1]: [t_time_sk#74] Join type: Inner Join condition: None -(140) Project [codegen id : 28] +(130) Project [codegen id : 28] Output [1]: [ss_store_sk#71] Input [3]: [ss_sold_time_sk#69, ss_store_sk#71, t_time_sk#74] -(141) ReusedExchange [Reuses operator id: 23] +(131) ReusedExchange [Reuses operator id: 23] Output [1]: [s_store_sk#77] -(142) BroadcastHashJoin [codegen id : 28] +(132) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_store_sk#71] Right keys [1]: [s_store_sk#77] Join type: Inner Join condition: None -(143) Project [codegen id : 28] +(133) Project [codegen id : 28] Output: [] Input [2]: [ss_store_sk#71, s_store_sk#77] -(144) HashAggregate [codegen id : 28] +(134) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#78] Results [1]: [count#79] -(145) RowToColumnar -Input [1]: [count#79] - -(146) CometColumnarExchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] - -(147) ColumnarToRow [codegen id : 29] +(135) Exchange Input [1]: [count#79] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] -(148) HashAggregate [codegen id : 29] +(136) HashAggregate [codegen id : 29] Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#80] Results [1]: [count(1)#80 AS h11_to_11_30#81] -(149) BroadcastExchange +(137) BroadcastExchange Input [1]: [h11_to_11_30#81] Arguments: IdentityBroadcastMode, [plan_id=19] -(150) BroadcastNestedLoopJoin [codegen id : 40] +(138) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None @@ -884,27 +832,27 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(152) CometFilter +(140) CometFilter Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] Condition : ((isnotnull(ss_hdemo_sk#83) AND isnotnull(ss_sold_time_sk#82)) AND isnotnull(ss_store_sk#84)) -(153) CometProject +(141) CometProject Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] Arguments: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84], [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] -(154) ColumnarToRow [codegen id : 33] +(142) ColumnarToRow [codegen id : 33] Input [3]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] -(155) ReusedExchange [Reuses operator id: 9] +(143) ReusedExchange [Reuses operator id: 9] Output [1]: [hd_demo_sk#86] -(156) BroadcastHashJoin [codegen id : 33] +(144) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_hdemo_sk#83] Right keys [1]: [hd_demo_sk#86] Join type: Inner Join condition: None -(157) Project [codegen id : 33] +(145) Project [codegen id : 33] Output [2]: [ss_sold_time_sk#82, ss_store_sk#84] Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, hd_demo_sk#86] @@ -915,73 +863,67 @@ Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(159) CometFilter +(147) CometFilter Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] Condition : ((((isnotnull(t_hour#88) AND isnotnull(t_minute#89)) AND (t_hour#88 = 11)) AND (t_minute#89 >= 30)) AND isnotnull(t_time_sk#87)) -(160) CometProject +(148) CometProject Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] Arguments: [t_time_sk#87], [t_time_sk#87] -(161) ColumnarToRow [codegen id : 31] +(149) ColumnarToRow [codegen id : 31] Input [1]: [t_time_sk#87] -(162) BroadcastExchange +(150) BroadcastExchange Input [1]: [t_time_sk#87] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] -(163) BroadcastHashJoin [codegen id : 33] +(151) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_sold_time_sk#82] Right keys [1]: [t_time_sk#87] Join type: Inner Join condition: None -(164) Project [codegen id : 33] +(152) Project [codegen id : 33] Output [1]: [ss_store_sk#84] Input [3]: [ss_sold_time_sk#82, ss_store_sk#84, t_time_sk#87] -(165) ReusedExchange [Reuses operator id: 23] +(153) ReusedExchange [Reuses operator id: 23] Output [1]: [s_store_sk#90] -(166) BroadcastHashJoin [codegen id : 33] +(154) BroadcastHashJoin [codegen id : 33] Left keys [1]: [ss_store_sk#84] Right keys [1]: [s_store_sk#90] Join type: Inner Join condition: None -(167) Project [codegen id : 33] +(155) Project [codegen id : 33] Output: [] Input [2]: [ss_store_sk#84, s_store_sk#90] -(168) HashAggregate [codegen id : 33] +(156) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#91] Results [1]: [count#92] -(169) RowToColumnar -Input [1]: [count#92] - -(170) CometColumnarExchange -Input [1]: [count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=21] - -(171) ColumnarToRow [codegen id : 34] +(157) Exchange Input [1]: [count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=21] -(172) HashAggregate [codegen id : 34] +(158) HashAggregate [codegen id : 34] Input [1]: [count#92] Keys: [] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#93] Results [1]: [count(1)#93 AS h11_30_to_12#94] -(173) BroadcastExchange +(159) BroadcastExchange Input [1]: [h11_30_to_12#94] Arguments: IdentityBroadcastMode, [plan_id=22] -(174) BroadcastNestedLoopJoin [codegen id : 40] +(160) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None @@ -992,27 +934,27 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(176) CometFilter +(162) CometFilter Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] Condition : ((isnotnull(ss_hdemo_sk#96) AND isnotnull(ss_sold_time_sk#95)) AND isnotnull(ss_store_sk#97)) -(177) CometProject +(163) CometProject Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] Arguments: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97], [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] -(178) ColumnarToRow [codegen id : 38] +(164) ColumnarToRow [codegen id : 38] Input [3]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] -(179) ReusedExchange [Reuses operator id: 9] +(165) ReusedExchange [Reuses operator id: 9] Output [1]: [hd_demo_sk#99] -(180) BroadcastHashJoin [codegen id : 38] +(166) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_hdemo_sk#96] Right keys [1]: [hd_demo_sk#99] Join type: Inner Join condition: None -(181) Project [codegen id : 38] +(167) Project [codegen id : 38] Output [2]: [ss_sold_time_sk#95, ss_store_sk#97] Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, hd_demo_sk#99] @@ -1023,73 +965,67 @@ Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(183) CometFilter +(169) CometFilter Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] Condition : ((((isnotnull(t_hour#101) AND isnotnull(t_minute#102)) AND (t_hour#101 = 12)) AND (t_minute#102 < 30)) AND isnotnull(t_time_sk#100)) -(184) CometProject +(170) CometProject Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] Arguments: [t_time_sk#100], [t_time_sk#100] -(185) ColumnarToRow [codegen id : 36] +(171) ColumnarToRow [codegen id : 36] Input [1]: [t_time_sk#100] -(186) BroadcastExchange +(172) BroadcastExchange Input [1]: [t_time_sk#100] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23] -(187) BroadcastHashJoin [codegen id : 38] +(173) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_time_sk#95] Right keys [1]: [t_time_sk#100] Join type: Inner Join condition: None -(188) Project [codegen id : 38] +(174) Project [codegen id : 38] Output [1]: [ss_store_sk#97] Input [3]: [ss_sold_time_sk#95, ss_store_sk#97, t_time_sk#100] -(189) ReusedExchange [Reuses operator id: 23] +(175) ReusedExchange [Reuses operator id: 23] Output [1]: [s_store_sk#103] -(190) BroadcastHashJoin [codegen id : 38] +(176) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_store_sk#97] Right keys [1]: [s_store_sk#103] Join type: Inner Join condition: None -(191) Project [codegen id : 38] +(177) Project [codegen id : 38] Output: [] Input [2]: [ss_store_sk#97, s_store_sk#103] -(192) HashAggregate [codegen id : 38] +(178) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#104] Results [1]: [count#105] -(193) RowToColumnar -Input [1]: [count#105] - -(194) CometColumnarExchange -Input [1]: [count#105] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=24] - -(195) ColumnarToRow [codegen id : 39] +(179) Exchange Input [1]: [count#105] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=24] -(196) HashAggregate [codegen id : 39] +(180) HashAggregate [codegen id : 39] Input [1]: [count#105] Keys: [] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#106] Results [1]: [count(1)#106 AS h12_to_12_30#107] -(197) BroadcastExchange +(181) BroadcastExchange Input [1]: [h12_to_12_30#107] Arguments: IdentityBroadcastMode, [plan_id=25] -(198) BroadcastNestedLoopJoin [codegen id : 40] +(182) BroadcastNestedLoopJoin [codegen id : 40] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index 85d520e7d5..b497e0bab6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -7,122 +7,52 @@ WholeStageCodegen (40) BroadcastNestedLoopJoin BroadcastNestedLoopJoin HashAggregate [count] [count(1),h8_30_to_9,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + InputAdapter + Exchange #1 + WholeStageCodegen (4) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_store_name,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 WholeStageCodegen (9) HashAggregate [count] [count(1),h9_to_9_30,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #6 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (14) - HashAggregate [count] [count(1),h9_30_to_10,count] - ColumnarToRow InputAdapter - CometColumnarExchange #9 - RowToColumnar - WholeStageCodegen (13) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (19) - HashAggregate [count] [count(1),h10_to_10_30,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #12 - RowToColumnar - WholeStageCodegen (18) + Exchange #6 + WholeStageCodegen (8) HashAggregate [count,count] Project BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -138,8 +68,8 @@ WholeStageCodegen (40) InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (16) + BroadcastExchange #7 + WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [t_time_sk] @@ -147,15 +77,13 @@ WholeStageCodegen (40) CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (24) - HashAggregate [count] [count(1),h10_30_to_11,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #15 - RowToColumnar - WholeStageCodegen (23) + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (14) + HashAggregate [count] [count(1),h9_30_to_10,count] + InputAdapter + Exchange #9 + WholeStageCodegen (13) HashAggregate [count,count] Project BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -171,8 +99,8 @@ WholeStageCodegen (40) InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (21) + BroadcastExchange #10 + WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [t_time_sk] @@ -180,15 +108,13 @@ WholeStageCodegen (40) CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (29) - HashAggregate [count] [count(1),h11_to_11_30,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #18 - RowToColumnar - WholeStageCodegen (28) + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (19) + HashAggregate [count] [count(1),h10_to_10_30,count] + InputAdapter + Exchange #12 + WholeStageCodegen (18) HashAggregate [count,count] Project BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -204,8 +130,8 @@ WholeStageCodegen (40) InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter - BroadcastExchange #19 - WholeStageCodegen (26) + BroadcastExchange #13 + WholeStageCodegen (16) ColumnarToRow InputAdapter CometProject [t_time_sk] @@ -213,15 +139,13 @@ WholeStageCodegen (40) CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (34) - HashAggregate [count] [count(1),h11_30_to_12,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #21 - RowToColumnar - WholeStageCodegen (33) + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (24) + HashAggregate [count] [count(1),h10_30_to_11,count] + InputAdapter + Exchange #15 + WholeStageCodegen (23) HashAggregate [count,count] Project BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -237,8 +161,8 @@ WholeStageCodegen (40) InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter - BroadcastExchange #22 - WholeStageCodegen (31) + BroadcastExchange #16 + WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [t_time_sk] @@ -246,15 +170,13 @@ WholeStageCodegen (40) CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (39) - HashAggregate [count] [count(1),h12_to_12_30,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #24 - RowToColumnar - WholeStageCodegen (38) + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (29) + HashAggregate [count] [count(1),h11_to_11_30,count] + InputAdapter + Exchange #18 + WholeStageCodegen (28) HashAggregate [count,count] Project BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -270,8 +192,8 @@ WholeStageCodegen (40) InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter - BroadcastExchange #25 - WholeStageCodegen (36) + BroadcastExchange #19 + WholeStageCodegen (26) ColumnarToRow InputAdapter CometProject [t_time_sk] @@ -279,3 +201,65 @@ WholeStageCodegen (40) CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #20 + WholeStageCodegen (34) + HashAggregate [count] [count(1),h11_30_to_12,count] + InputAdapter + Exchange #21 + WholeStageCodegen (33) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #22 + WholeStageCodegen (31) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #23 + WholeStageCodegen (39) + HashAggregate [count] [count(1),h12_to_12_30,count] + InputAdapter + Exchange #24 + WholeStageCodegen (38) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #25 + WholeStageCodegen (36) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index d7834e74c8..a59560090d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -1,35 +1,31 @@ == Physical Plan == -TakeOrderedAndProject (31) -+- * Project (30) - +- * Filter (29) - +- Window (28) - +- * ColumnarToRow (27) - +- CometSort (26) - +- CometColumnarExchange (25) - +- RowToColumnar (24) - +- * HashAggregate (23) - +- * ColumnarToRow (22) - +- CometColumnarExchange (21) - +- RowToColumnar (20) - +- * HashAggregate (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.store_sales (4) - : +- ReusedExchange (10) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store (13) +TakeOrderedAndProject (27) ++- * Project (26) + +- * Filter (25) + +- Window (24) + +- * Sort (23) + +- Exchange (22) + +- * HashAggregate (21) + +- Exchange (20) + +- * HashAggregate (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.store_sales (4) + : +- ReusedExchange (10) + +- BroadcastExchange (16) + +- * ColumnarToRow (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.store (13) (unknown) Scan parquet spark_catalog.default.item @@ -75,7 +71,7 @@ Join condition: None Output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] Input [8]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] -(10) ReusedExchange [Reuses operator id: 36] +(10) ReusedExchange [Reuses operator id: 32] Output [2]: [d_date_sk#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -123,61 +119,49 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#7))] Aggregate Attributes [1]: [sum#15] Results [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] -(20) RowToColumnar +(20) Exchange Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] +Arguments: hashpartitioning(i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometColumnarExchange -Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) ColumnarToRow [codegen id : 5] -Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] - -(23) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 5] Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#7))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#7))#17] Results [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS _w0#19] -(24) RowToColumnar -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] - -(25) CometColumnarExchange -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometSort +(22) Exchange Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -Arguments: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19], [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST] +Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(27) ColumnarToRow [codegen id : 6] +(23) Sort [codegen id : 6] Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] +Arguments: [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST], false, 0 -(28) Window +(24) Window Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#4, i_brand#2, s_store_name#13, s_company_name#14] -(29) Filter [codegen id : 7] +(25) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] Condition : CASE WHEN NOT (avg_monthly_sales#20 = 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END -(30) Project [codegen id : 7] +(26) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] -(31) TakeOrderedAndProject +(27) TakeOrderedAndProject Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] Arguments: 100, [(sum_sales#18 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) +BroadcastExchange (32) ++- * ColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan parquet spark_catalog.default.date_dim (28) (unknown) Scan parquet spark_catalog.default.date_dim @@ -187,18 +171,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(33) CometFilter +(29) CometFilter Input [3]: [d_date_sk#10, d_year#21, d_moy#11] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 1999)) AND isnotnull(d_date_sk#10)) -(34) CometProject +(30) CometProject Input [3]: [d_date_sk#10, d_year#21, d_moy#11] Arguments: [d_date_sk#10, d_moy#11], [d_date_sk#10, d_moy#11] -(35) ColumnarToRow [codegen id : 1] +(31) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_moy#11] -(36) BroadcastExchange +(32) BroadcastExchange Input [2]: [d_date_sk#10, d_moy#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index 38e228738e..bb9e4e17e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -5,50 +5,46 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (6) - ColumnarToRow + Sort [i_category,i_brand,s_store_name,s_company_name] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_category,i_class,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_category,i_class,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index f277fc9feb..4fb9fc546a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -27,9 +27,9 @@ Input [1]: [r_reason_sk#1] Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] * Project (12) -+- * ColumnarToRow (11) - +- CometHashAggregate (10) - +- CometColumnarExchange (9) ++- * HashAggregate (11) + +- Exchange (10) + +- * ColumnarToRow (9) +- CometHashAggregate (8) +- CometProject (7) +- CometFilter (6) @@ -56,21 +56,23 @@ Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] -(9) CometColumnarExchange +(9) ColumnarToRow [codegen id : 1] Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) CometHashAggregate +(10) Exchange +Input [5]: [count#21, sum#22, count#23, sum#24, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] + +(11) HashAggregate [codegen id : 2] Input [5]: [count#21, sum#22, count#23, sum#24, count#25] Keys: [] Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] +Aggregate Attributes [3]: [count(1)#26, avg(UnscaledValue(ss_ext_discount_amt#18))#27, avg(UnscaledValue(ss_net_paid#19))#28] +Results [3]: [count(1)#26 AS count(1)#29, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#27 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#30, cast((avg(UnscaledValue(ss_net_paid#19))#28 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#31] -(11) ColumnarToRow [codegen id : 1] -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] - -(12) Project [codegen id : 1] -Output [1]: [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] +(12) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#29, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#30, avg(ss_net_paid), avg(ss_net_paid)#31) AS mergedValue#32] +Input [3]: [count(1)#29, avg(ss_ext_discount_amt)#30, avg(ss_net_paid)#31] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] @@ -78,9 +80,9 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] * Project (20) -+- * ColumnarToRow (19) - +- CometHashAggregate (18) - +- CometColumnarExchange (17) ++- * HashAggregate (19) + +- Exchange (18) + +- * ColumnarToRow (17) +- CometHashAggregate (16) +- CometProject (15) +- CometFilter (14) @@ -88,40 +90,42 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] +Output [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (14) CometFilter -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) +Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] +Condition : ((isnotnull(ss_quantity#33) AND (ss_quantity#33 >= 21)) AND (ss_quantity#33 <= 40)) (15) CometProject -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] +Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] +Arguments: [ss_ext_discount_amt#34, ss_net_paid#35], [ss_ext_discount_amt#34, ss_net_paid#35] (16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] +Input [2]: [ss_ext_discount_amt#34, ss_net_paid#35] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#34)), partial_avg(UnscaledValue(ss_net_paid#35))] -(17) CometColumnarExchange -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(17) ColumnarToRow [codegen id : 1] +Input [5]: [count#37, sum#38, count#39, sum#40, count#41] -(18) CometHashAggregate -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] +(18) Exchange +Input [5]: [count#37, sum#38, count#39, sum#40, count#41] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(19) ColumnarToRow [codegen id : 1] -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] +(19) HashAggregate [codegen id : 2] +Input [5]: [count#37, sum#38, count#39, sum#40, count#41] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#34)), avg(UnscaledValue(ss_net_paid#35))] +Aggregate Attributes [3]: [count(1)#42, avg(UnscaledValue(ss_ext_discount_amt#34))#43, avg(UnscaledValue(ss_net_paid#35))#44] +Results [3]: [count(1)#42 AS count(1)#45, cast((avg(UnscaledValue(ss_ext_discount_amt#34))#43 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#46, cast((avg(UnscaledValue(ss_net_paid#35))#44 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#47] -(20) Project [codegen id : 1] -Output [1]: [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] +(20) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#45, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#46, avg(ss_net_paid), avg(ss_net_paid)#47) AS mergedValue#48] +Input [3]: [count(1)#45, avg(ss_ext_discount_amt)#46, avg(ss_net_paid)#47] Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] @@ -129,9 +133,9 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] * Project (28) -+- * ColumnarToRow (27) - +- CometHashAggregate (26) - +- CometColumnarExchange (25) ++- * HashAggregate (27) + +- Exchange (26) + +- * ColumnarToRow (25) +- CometHashAggregate (24) +- CometProject (23) +- CometFilter (22) @@ -139,40 +143,42 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] +Output [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (22) CometFilter -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) +Input [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] +Condition : ((isnotnull(ss_quantity#49) AND (ss_quantity#49 >= 41)) AND (ss_quantity#49 <= 60)) (23) CometProject -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] +Input [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] +Arguments: [ss_ext_discount_amt#50, ss_net_paid#51], [ss_ext_discount_amt#50, ss_net_paid#51] (24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] +Input [2]: [ss_ext_discount_amt#50, ss_net_paid#51] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#50)), partial_avg(UnscaledValue(ss_net_paid#51))] -(25) CometColumnarExchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(25) ColumnarToRow [codegen id : 1] +Input [5]: [count#53, sum#54, count#55, sum#56, count#57] -(26) CometHashAggregate -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] +(26) Exchange +Input [5]: [count#53, sum#54, count#55, sum#56, count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(27) ColumnarToRow [codegen id : 1] -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] +(27) HashAggregate [codegen id : 2] +Input [5]: [count#53, sum#54, count#55, sum#56, count#57] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#50)), avg(UnscaledValue(ss_net_paid#51))] +Aggregate Attributes [3]: [count(1)#58, avg(UnscaledValue(ss_ext_discount_amt#50))#59, avg(UnscaledValue(ss_net_paid#51))#60] +Results [3]: [count(1)#58 AS count(1)#61, cast((avg(UnscaledValue(ss_ext_discount_amt#50))#59 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#62, cast((avg(UnscaledValue(ss_net_paid#51))#60 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] -(28) Project [codegen id : 1] -Output [1]: [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] +(28) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#61, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#62, avg(ss_net_paid), avg(ss_net_paid)#63) AS mergedValue#64] +Input [3]: [count(1)#61, avg(ss_ext_discount_amt)#62, avg(ss_net_paid)#63] Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] @@ -180,9 +186,9 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * Project (36) -+- * ColumnarToRow (35) - +- CometHashAggregate (34) - +- CometColumnarExchange (33) ++- * HashAggregate (35) + +- Exchange (34) + +- * ColumnarToRow (33) +- CometHashAggregate (32) +- CometProject (31) +- CometFilter (30) @@ -190,40 +196,42 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] +Output [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (30) CometFilter -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) +Input [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] +Condition : ((isnotnull(ss_quantity#65) AND (ss_quantity#65 >= 61)) AND (ss_quantity#65 <= 80)) (31) CometProject -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] +Input [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] +Arguments: [ss_ext_discount_amt#66, ss_net_paid#67], [ss_ext_discount_amt#66, ss_net_paid#67] (32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] +Input [2]: [ss_ext_discount_amt#66, ss_net_paid#67] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#66)), partial_avg(UnscaledValue(ss_net_paid#67))] -(33) CometColumnarExchange -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(33) ColumnarToRow [codegen id : 1] +Input [5]: [count#69, sum#70, count#71, sum#72, count#73] -(34) CometHashAggregate -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] +(34) Exchange +Input [5]: [count#69, sum#70, count#71, sum#72, count#73] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(35) ColumnarToRow [codegen id : 1] -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] +(35) HashAggregate [codegen id : 2] +Input [5]: [count#69, sum#70, count#71, sum#72, count#73] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#66)), avg(UnscaledValue(ss_net_paid#67))] +Aggregate Attributes [3]: [count(1)#74, avg(UnscaledValue(ss_ext_discount_amt#66))#75, avg(UnscaledValue(ss_net_paid#67))#76] +Results [3]: [count(1)#74 AS count(1)#77, cast((avg(UnscaledValue(ss_ext_discount_amt#66))#75 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#78, cast((avg(UnscaledValue(ss_net_paid#67))#76 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#79] -(36) Project [codegen id : 1] -Output [1]: [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] +(36) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#77, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#78, avg(ss_net_paid), avg(ss_net_paid)#79) AS mergedValue#80] +Input [3]: [count(1)#77, avg(ss_ext_discount_amt)#78, avg(ss_net_paid)#79] Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] @@ -231,9 +239,9 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] * Project (44) -+- * ColumnarToRow (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) ++- * HashAggregate (43) + +- Exchange (42) + +- * ColumnarToRow (41) +- CometHashAggregate (40) +- CometProject (39) +- CometFilter (38) @@ -241,40 +249,42 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] +Output [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (38) CometFilter -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) +Input [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] +Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 81)) AND (ss_quantity#81 <= 100)) (39) CometProject -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] +Input [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] +Arguments: [ss_ext_discount_amt#82, ss_net_paid#83], [ss_ext_discount_amt#82, ss_net_paid#83] (40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] +Input [2]: [ss_ext_discount_amt#82, ss_net_paid#83] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#82)), partial_avg(UnscaledValue(ss_net_paid#83))] -(41) CometColumnarExchange -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(41) ColumnarToRow [codegen id : 1] +Input [5]: [count#85, sum#86, count#87, sum#88, count#89] -(42) CometHashAggregate -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] +(42) Exchange +Input [5]: [count#85, sum#86, count#87, sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(43) ColumnarToRow [codegen id : 1] -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] +(43) HashAggregate [codegen id : 2] +Input [5]: [count#85, sum#86, count#87, sum#88, count#89] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#82)), avg(UnscaledValue(ss_net_paid#83))] +Aggregate Attributes [3]: [count(1)#90, avg(UnscaledValue(ss_ext_discount_amt#82))#91, avg(UnscaledValue(ss_net_paid#83))#92] +Results [3]: [count(1)#90 AS count(1)#93, cast((avg(UnscaledValue(ss_ext_discount_amt#82))#91 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#94, cast((avg(UnscaledValue(ss_net_paid#83))#92 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#95] -(44) Project [codegen id : 1] -Output [1]: [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] +(44) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#93, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#94, avg(ss_net_paid), avg(ss_net_paid)#95) AS mergedValue#96] +Input [3]: [count(1)#93, avg(ss_ext_discount_amt)#94, avg(ss_net_paid)#95] Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index eb47e062d4..c54606f6e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -1,68 +1,78 @@ WholeStageCodegen (1) Project Subquery #1 - WholeStageCodegen (1) + WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - ColumnarToRow + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] InputAdapter - CometHashAggregate [count,sum,count,sum,count] - CometColumnarExchange #1 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Exchange #1 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 - WholeStageCodegen (1) + WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - ColumnarToRow + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] InputAdapter - CometHashAggregate [count,sum,count,sum,count] - CometColumnarExchange #2 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Exchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 - WholeStageCodegen (1) + WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - ColumnarToRow + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] InputAdapter - CometHashAggregate [count,sum,count,sum,count] - CometColumnarExchange #3 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Exchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 - WholeStageCodegen (1) + WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - ColumnarToRow + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] InputAdapter - CometHashAggregate [count,sum,count,sum,count] - CometColumnarExchange #4 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Exchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 - WholeStageCodegen (1) + WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - ColumnarToRow + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] InputAdapter - CometHashAggregate [count,sum,count,sum,count] - CometColumnarExchange #5 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Exchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 ColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt index 8931917005..bb9bf128e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt @@ -1,59 +1,55 @@ == Physical Plan == -* Project (55) -+- * BroadcastNestedLoopJoin Inner BuildRight (54) - :- * HashAggregate (30) - : +- * ColumnarToRow (29) - : +- CometColumnarExchange (28) - : +- RowToColumnar (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (11) - : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : :- * ColumnarToRow (4) - : : : : +- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- BroadcastExchange (9) - : : : +- * ColumnarToRow (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.household_demographics (5) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan parquet spark_catalog.default.time_dim (12) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.web_page (19) - +- BroadcastExchange (53) - +- * HashAggregate (52) - +- * ColumnarToRow (51) - +- CometColumnarExchange (50) - +- RowToColumnar (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * ColumnarToRow (34) - : : : +- CometProject (33) - : : : +- CometFilter (32) - : : : +- CometScan parquet spark_catalog.default.web_sales (31) - : : +- ReusedExchange (35) - : +- BroadcastExchange (42) - : +- * ColumnarToRow (41) - : +- CometProject (40) - : +- CometFilter (39) - : +- CometScan parquet spark_catalog.default.time_dim (38) - +- ReusedExchange (45) +* Project (51) ++- * BroadcastNestedLoopJoin Inner BuildRight (50) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (11) + : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : :- * ColumnarToRow (4) + : : : : +- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- BroadcastExchange (9) + : : : +- * ColumnarToRow (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.household_demographics (5) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometProject (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.time_dim (12) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometProject (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.web_page (19) + +- BroadcastExchange (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * ColumnarToRow (32) + : : : +- CometProject (31) + : : : +- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.web_sales (29) + : : +- ReusedExchange (33) + : +- BroadcastExchange (40) + : +- * ColumnarToRow (39) + : +- CometProject (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.time_dim (36) + +- ReusedExchange (43) (unknown) Scan parquet spark_catalog.default.web_sales @@ -177,17 +173,11 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#11] Results [1]: [count#12] -(27) RowToColumnar +(27) Exchange Input [1]: [count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(28) CometColumnarExchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) ColumnarToRow [codegen id : 10] -Input [1]: [count#12] - -(30) HashAggregate [codegen id : 10] +(28) HashAggregate [codegen id : 10] Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] @@ -201,27 +191,27 @@ Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct -(32) CometFilter +(30) CometFilter Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] Condition : ((isnotnull(ws_ship_hdemo_sk#16) AND isnotnull(ws_sold_time_sk#15)) AND isnotnull(ws_web_page_sk#17)) -(33) CometProject +(31) CometProject Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] Arguments: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17], [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] -(34) ColumnarToRow [codegen id : 8] +(32) ColumnarToRow [codegen id : 8] Input [3]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] -(35) ReusedExchange [Reuses operator id: 9] +(33) ReusedExchange [Reuses operator id: 9] Output [1]: [hd_demo_sk#19] -(36) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_ship_hdemo_sk#16] Right keys [1]: [hd_demo_sk#19] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(35) Project [codegen id : 8] Output [2]: [ws_sold_time_sk#15, ws_web_page_sk#17] Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, hd_demo_sk#19] @@ -232,77 +222,71 @@ Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct -(39) CometFilter +(37) CometFilter Input [2]: [t_time_sk#20, t_hour#21] Condition : (((isnotnull(t_hour#21) AND (t_hour#21 >= 19)) AND (t_hour#21 <= 20)) AND isnotnull(t_time_sk#20)) -(40) CometProject +(38) CometProject Input [2]: [t_time_sk#20, t_hour#21] Arguments: [t_time_sk#20], [t_time_sk#20] -(41) ColumnarToRow [codegen id : 6] +(39) ColumnarToRow [codegen id : 6] Input [1]: [t_time_sk#20] -(42) BroadcastExchange +(40) BroadcastExchange Input [1]: [t_time_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 8] +(41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_time_sk#15] Right keys [1]: [t_time_sk#20] Join type: Inner Join condition: None -(44) Project [codegen id : 8] +(42) Project [codegen id : 8] Output [1]: [ws_web_page_sk#17] Input [3]: [ws_sold_time_sk#15, ws_web_page_sk#17, t_time_sk#20] -(45) ReusedExchange [Reuses operator id: 23] +(43) ReusedExchange [Reuses operator id: 23] Output [1]: [wp_web_page_sk#22] -(46) BroadcastHashJoin [codegen id : 8] +(44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_web_page_sk#17] Right keys [1]: [wp_web_page_sk#22] Join type: Inner Join condition: None -(47) Project [codegen id : 8] +(45) Project [codegen id : 8] Output: [] Input [2]: [ws_web_page_sk#17, wp_web_page_sk#22] -(48) HashAggregate [codegen id : 8] +(46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#23] Results [1]: [count#24] -(49) RowToColumnar -Input [1]: [count#24] - -(50) CometColumnarExchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(51) ColumnarToRow [codegen id : 9] +(47) Exchange Input [1]: [count#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(52) HashAggregate [codegen id : 9] +(48) HashAggregate [codegen id : 9] Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#25] Results [1]: [count(1)#25 AS pmc#26] -(53) BroadcastExchange +(49) BroadcastExchange Input [1]: [pmc#26] Arguments: IdentityBroadcastMode, [plan_id=7] -(54) BroadcastNestedLoopJoin [codegen id : 10] +(50) BroadcastNestedLoopJoin [codegen id : 10] Join type: Inner Join condition: None -(55) Project [codegen id : 10] +(51) Project [codegen id : 10] Output [1]: [(cast(amc#14 as decimal(15,4)) / cast(pmc#26 as decimal(15,4))) AS am_pm_ratio#27] Input [2]: [amc#14, pmc#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index d9cfe34cab..c4e04b06bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -2,77 +2,73 @@ WholeStageCodegen (10) Project [amc,pmc] BroadcastNestedLoopJoin HashAggregate [count] [count(1),amc,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + InputAdapter + Exchange #1 + WholeStageCodegen (4) + HashAggregate [count,count] + Project + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_web_page_sk] + BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] + CometProject [t_time_sk] + CometFilter [t_hour,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [wp_web_page_sk] + CometFilter [wp_char_count,wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 WholeStageCodegen (9) HashAggregate [count] [count(1),pmc,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #6 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [count,count] - Project - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 + InputAdapter + Exchange #6 + WholeStageCodegen (8) + HashAggregate [count,count] + Project + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_web_page_sk] + BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + ReusedExchange [hd_demo_sk] #2 InputAdapter - ReusedExchange [wp_web_page_sk] #4 + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + InputAdapter + ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt index 0a57a5b1f4..c8110434e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt @@ -1,51 +1,47 @@ == Physical Plan == -* ColumnarToRow (47) -+- CometSort (46) - +- CometColumnarExchange (45) - +- RowToColumnar (44) - +- * HashAggregate (43) - +- * ColumnarToRow (42) - +- CometColumnarExchange (41) - +- RowToColumnar (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * Project (12) - : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.call_center (1) - : : : : : +- BroadcastExchange (7) - : : : : : +- * ColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (4) - : : : : +- ReusedExchange (10) - : : : +- BroadcastExchange (16) - : : : +- * ColumnarToRow (15) - : : : +- CometFilter (14) - : : : +- CometScan parquet spark_catalog.default.customer (13) - : : +- BroadcastExchange (23) - : : +- * ColumnarToRow (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan parquet spark_catalog.default.customer_address (19) - : +- BroadcastExchange (29) - : +- * ColumnarToRow (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.customer_demographics (26) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.household_demographics (32) +* Sort (43) ++- Exchange (42) + +- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (18) + : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : :- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * Project (9) + : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.call_center (1) + : : : : : +- BroadcastExchange (7) + : : : : : +- * ColumnarToRow (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (4) + : : : : +- ReusedExchange (10) + : : : +- BroadcastExchange (16) + : : : +- * ColumnarToRow (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.customer (13) + : : +- BroadcastExchange (23) + : : +- * ColumnarToRow (22) + : : +- CometProject (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.customer_address (19) + : +- BroadcastExchange (29) + : +- * ColumnarToRow (28) + : +- CometFilter (27) + : +- CometScan parquet spark_catalog.default.customer_demographics (26) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.household_demographics (32) (unknown) Scan parquet spark_catalog.default.call_center @@ -91,7 +87,7 @@ Join condition: None Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8] Input [8]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] -(10) ReusedExchange [Reuses operator id: 52] +(10) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#10] (11) BroadcastHashJoin [codegen id : 7] @@ -231,45 +227,33 @@ Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#7))] Aggregate Attributes [1]: [sum#22] Results [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] -(40) RowToColumnar +(40) Exchange Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] +Arguments: hashpartitioning(cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) CometColumnarExchange -Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] -Arguments: hashpartitioning(cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) ColumnarToRow [codegen id : 8] -Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] - -(43) HashAggregate [codegen id : 8] +(41) HashAggregate [codegen id : 8] Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19] Functions [1]: [sum(UnscaledValue(cr_net_loss#7))] Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#7))#24] Results [4]: [cc_call_center_id#2 AS Call_Center#25, cc_name#3 AS Call_Center_Name#26, cc_manager#4 AS Manager#27, MakeDecimal(sum(UnscaledValue(cr_net_loss#7))#24,17,2) AS Returns_Loss#28] -(44) RowToColumnar -Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] - -(45) CometColumnarExchange -Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] -Arguments: rangepartitioning(Returns_Loss#28 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(46) CometSort +(42) Exchange Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] -Arguments: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28], [Returns_Loss#28 DESC NULLS LAST] +Arguments: rangepartitioning(Returns_Loss#28 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(47) ColumnarToRow [codegen id : 9] +(43) Sort [codegen id : 9] Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] +Arguments: [Returns_Loss#28 DESC NULLS LAST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) (unknown) Scan parquet spark_catalog.default.date_dim @@ -279,18 +263,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter +(45) CometFilter Input [3]: [d_date_sk#10, d_year#29, d_moy#30] Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 1998)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#10)) -(50) CometProject +(46) CometProject Input [3]: [d_date_sk#10, d_year#29, d_moy#30] Arguments: [d_date_sk#10], [d_date_sk#10] -(51) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(52) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index 75273fafec..e5d62e3c0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -1,77 +1,73 @@ WholeStageCodegen (9) - ColumnarToRow + Sort [Returns_Loss] InputAdapter - CometSort [Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - ColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cr_call_center_sk,cr_returning_customer_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 + Exchange [Returns_Loss] #1 + WholeStageCodegen (8) + HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] + InputAdapter + Exchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 + WholeStageCodegen (7) + HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] + BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] + ColumnarToRow + InputAdapter + CometFilter [cc_call_center_sk] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) + BroadcastExchange #6 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index 04fa7a1bac..111c30960b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -1,37 +1,33 @@ == Physical Plan == -* HashAggregate (33) -+- * ColumnarToRow (32) - +- CometColumnarExchange (31) - +- RowToColumnar (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (23) - : +- * Filter (22) - : +- * HashAggregate (21) - : +- * ColumnarToRow (20) - : +- CometColumnarExchange (19) - : +- RowToColumnar (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * ColumnarToRow (13) - : : +- CometFilter (12) - : : +- CometScan parquet spark_catalog.default.web_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (26) +* HashAggregate (29) ++- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (23) + : +- * BroadcastHashJoin Inner BuildRight (22) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.item (4) + : +- BroadcastExchange (21) + : +- * Filter (20) + : +- * HashAggregate (19) + : +- Exchange (18) + : +- * HashAggregate (17) + : +- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * ColumnarToRow (13) + : : +- CometFilter (12) + : : +- CometScan parquet spark_catalog.default.web_sales (11) + : +- ReusedExchange (14) + +- ReusedExchange (24) (unknown) Scan parquet spark_catalog.default.web_sales @@ -96,7 +92,7 @@ Condition : isnotnull(ws_item_sk#7) (13) ColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -(14) ReusedExchange [Reuses operator id: 38] +(14) ReusedExchange [Reuses operator id: 34] Output [1]: [d_date_sk#11] (15) BroadcastHashJoin [codegen id : 3] @@ -116,72 +112,60 @@ Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] Aggregate Attributes [2]: [sum#12, count#13] Results [3]: [ws_item_sk#7, sum#14, count#15] -(18) RowToColumnar +(18) Exchange Input [3]: [ws_item_sk#7, sum#14, count#15] +Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(19) CometColumnarExchange -Input [3]: [ws_item_sk#7, sum#14, count#15] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(20) ColumnarToRow [codegen id : 4] -Input [3]: [ws_item_sk#7, sum#14, count#15] - -(21) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 4] Input [3]: [ws_item_sk#7, sum#14, count#15] Keys [1]: [ws_item_sk#7] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#16] Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] -(22) Filter [codegen id : 4] +(20) Filter [codegen id : 4] Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) -(23) BroadcastExchange +(21) BroadcastExchange Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] Right keys [1]: [ws_item_sk#7] Join type: Inner Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) -(25) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] -(26) ReusedExchange [Reuses operator id: 38] +(24) ReusedExchange [Reuses operator id: 34] Output [1]: [d_date_sk#18] -(27) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#2] Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#18] -(29) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] Aggregate Attributes [1]: [sum#19] Results [1]: [sum#20] -(30) RowToColumnar -Input [1]: [sum#20] - -(31) CometColumnarExchange -Input [1]: [sum#20] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(32) ColumnarToRow [codegen id : 7] +(28) Exchange Input [1]: [sum#20] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(33) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 7] Input [1]: [sum#20] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] @@ -191,11 +175,11 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#21,17,2) AS ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (38) -+- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) (unknown) Scan parquet spark_catalog.default.date_dim @@ -205,18 +189,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(35) CometFilter +(31) CometFilter Input [2]: [d_date_sk#18, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) -(36) CometProject +(32) CometProject Input [2]: [d_date_sk#18, d_date#23] Arguments: [d_date_sk#18], [d_date_sk#18] -(37) ColumnarToRow [codegen id : 1] +(33) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#18] -(38) BroadcastExchange +(34) BroadcastExchange Input [1]: [d_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index 3b87e0d3ec..a5e724c1ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -1,56 +1,52 @@ WholeStageCodegen (7) HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [ws_ext_discount_amt] [sum,sum] - Project [ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] - Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow + InputAdapter + Exchange #1 + WholeStageCodegen (6) + HashAggregate [ws_ext_discount_amt] [sum,sum] + Project [ws_ext_discount_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_discount_amt,ws_sold_date_sk] + BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] + Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_ext_discount_amt] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Filter [(1.3 * avg(ws_ext_discount_amt))] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] InputAdapter - CometFilter [ws_item_sk,ws_ext_discount_amt] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + Exchange [ws_item_sk] #5 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] + Project [ws_item_sk,ws_ext_discount_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #5 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] - Project [ws_item_sk,ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 5cf504d2d6..ee17b52919 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -1,29 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- * ColumnarToRow (23) - +- CometColumnarExchange (22) - +- RowToColumnar (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * ColumnarToRow (12) - : +- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometColumnarExchange (3) - : : +- CometProject (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometColumnarExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_returns (5) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.reason (13) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * Project (20) + +- * BroadcastHashJoin Inner BuildRight (19) + :- * Project (13) + : +- * SortMergeJoin Inner (12) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * ColumnarToRow (3) + : : +- CometProject (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- * Sort (11) + : +- Exchange (10) + : +- * ColumnarToRow (9) + : +- CometProject (8) + : +- CometFilter (7) + : +- CometScan parquet spark_catalog.default.store_returns (6) + +- BroadcastExchange (18) + +- * ColumnarToRow (17) + +- CometProject (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.reason (14) (unknown) Scan parquet spark_catalog.default.store_sales @@ -36,13 +35,16 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) -(7) CometProject +(8) CometProject Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -(8) CometColumnarExchange +(9) ColumnarToRow [codegen id : 3] Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(9) CometSort +(10) Exchange Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(10) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner +(11) Sort [codegen id : 4] +Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] +Arguments: [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST], false, 0 -(11) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +(12) SortMergeJoin [codegen id : 6] +Left keys [2]: [ss_item_sk#1, ss_ticket_number#3] +Right keys [2]: [sr_item_sk#7, sr_ticket_number#9] +Join type: Inner +Join condition: None -(12) ColumnarToRow [codegen id : 2] -Input [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +(13) Project [codegen id : 6] +Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] (unknown) Scan parquet spark_catalog.default.reason Output [2]: [r_reason_sk#12, r_reason_desc#13] @@ -86,56 +89,50 @@ Location [not included in comparison]/{warehouse_dir}/reason] PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct -(14) CometFilter +(15) CometFilter Input [2]: [r_reason_sk#12, r_reason_desc#13] Condition : ((isnotnull(r_reason_desc#13) AND (r_reason_desc#13 = reason 28 )) AND isnotnull(r_reason_sk#12)) -(15) CometProject +(16) CometProject Input [2]: [r_reason_sk#12, r_reason_desc#13] Arguments: [r_reason_sk#12], [r_reason_sk#12] -(16) ColumnarToRow [codegen id : 1] +(17) ColumnarToRow [codegen id : 5] Input [1]: [r_reason_sk#12] -(17) BroadcastExchange +(18) BroadcastExchange Input [1]: [r_reason_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(18) BroadcastHashJoin [codegen id : 2] +(19) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_reason_sk#8] Right keys [1]: [r_reason_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 2] +(20) Project [codegen id : 6] Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -(20) HashAggregate [codegen id : 2] +(21) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#2, act_sales#14] Keys [1]: [ss_customer_sk#2] Functions [1]: [partial_sum(act_sales#14)] Aggregate Attributes [2]: [sum#15, isEmpty#16] Results [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -(21) RowToColumnar -Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] - -(22) CometColumnarExchange -Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(23) ColumnarToRow [codegen id : 3] +(22) Exchange Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(24) HashAggregate [codegen id : 3] +(23) HashAggregate [codegen id : 7] Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Keys [1]: [ss_customer_sk#2] Functions [1]: [sum(act_sales#14)] Aggregate Attributes [1]: [sum(act_sales#14)#19] Results [2]: [ss_customer_sk#2, sum(act_sales#14)#19 AS sumsales#20] -(25) TakeOrderedAndProject +(24) TakeOrderedAndProject Input [2]: [ss_customer_sk#2, sumsales#20] Arguments: 100, [sumsales#20 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#20] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index fbd6290f3e..3ec7ac7b6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -1,32 +1,40 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] - WholeStageCodegen (3) + WholeStageCodegen (7) HashAggregate [ss_customer_sk,sum,isEmpty] [sum(act_sales),sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_customer_sk] #1 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] - Project [ss_customer_sk,sr_return_quantity,ss_quantity,ss_sales_price] - BroadcastHashJoin [sr_reason_sk,r_reason_sk] - ColumnarToRow - InputAdapter - CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_ticket_number] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_ticket_number] - CometColumnarExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + InputAdapter + Exchange [ss_customer_sk] #1 + WholeStageCodegen (6) + HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] + Project [ss_customer_sk,sr_return_quantity,ss_quantity,ss_sales_price] + BroadcastHashJoin [sr_reason_sk,r_reason_sk] + Project [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [r_reason_sk] - CometFilter [r_reason_desc,r_reason_sk] - CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + WholeStageCodegen (2) + Sort [ss_item_sk,ss_ticket_number] + InputAdapter + Exchange [ss_item_sk,ss_ticket_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_item_sk,sr_ticket_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [r_reason_sk] + CometFilter [r_reason_desc,r_reason_sk] + CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index e48a84a6fe..1ed6d0cb3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -1,51 +1,49 @@ == Physical Plan == -* HashAggregate (47) -+- * ColumnarToRow (46) - +- CometColumnarExchange (45) - +- RowToColumnar (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometSort (5) - : : : : : +- CometColumnarExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- * ColumnarToRow (11) - : : : : +- CometSort (10) - : : : : +- CometColumnarExchange (9) - : : : : +- CometProject (8) - : : : : +- CometScan parquet spark_catalog.default.web_sales (7) - : : : +- * ColumnarToRow (18) - : : : +- CometSort (17) - : : : +- CometColumnarExchange (16) - : : : +- CometProject (15) - : : : +- CometScan parquet spark_catalog.default.web_returns (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.web_site (34) +* HashAggregate (45) ++- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- * SortMergeJoin LeftSemi (12) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * ColumnarToRow (9) + : : : : +- CometProject (8) + : : : : +- CometScan parquet spark_catalog.default.web_sales (7) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * ColumnarToRow (16) + : : : +- CometProject (15) + : : : +- CometScan parquet spark_catalog.default.web_returns (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometProject (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.date_dim (20) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.web_site (34) (unknown) Scan parquet spark_catalog.default.web_sales @@ -63,16 +61,16 @@ Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(4) CometColumnarExchange +(4) ColumnarToRow [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) CometSort +(5) Exchange Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(6) ColumnarToRow [codegen id : 1] +(6) Sort [codegen id : 2] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] @@ -84,24 +82,24 @@ ReadSchema: struct Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] -(9) CometColumnarExchange +(9) ColumnarToRow [codegen id : 3] Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) CometSort +(10) Exchange Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) ColumnarToRow [codegen id : 2] +(11) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: [ws_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 3] +(12) SortMergeJoin [codegen id : 5] Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#10] Join type: LeftSemi Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) -(13) Project [codegen id : 3] +(13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -115,18 +113,18 @@ ReadSchema: struct Input [2]: [wr_order_number#12, wr_returned_date_sk#13] Arguments: [wr_order_number#12], [wr_order_number#12] -(16) CometColumnarExchange +(16) ColumnarToRow [codegen id : 6] Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) CometSort +(17) Exchange Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] +Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(18) ColumnarToRow [codegen id : 4] +(18) Sort [codegen id : 7] Input [1]: [wr_order_number#12] +Arguments: [wr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 8] +(19) SortMergeJoin [codegen id : 11] Left keys [1]: [ws_order_number#5] Right keys [1]: [wr_order_number#12] Join type: LeftAnti @@ -147,20 +145,20 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(23) ColumnarToRow [codegen id : 5] +(23) ColumnarToRow [codegen id : 8] Input [1]: [d_date_sk#14] (24) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 8] +(25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 8] +(26) Project [codegen id : 11] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] @@ -179,20 +177,20 @@ Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = IL)) AND isnotnull(ca_ad Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) ColumnarToRow [codegen id : 6] +(30) ColumnarToRow [codegen id : 9] Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(33) Project [codegen id : 11] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] @@ -211,55 +209,49 @@ Condition : ((isnotnull(web_company_name#19) AND (web_company_name#19 = pri Input [2]: [web_site_sk#18, web_company_name#19] Arguments: [web_site_sk#18], [web_site_sk#18] -(37) ColumnarToRow [codegen id : 7] +(37) ColumnarToRow [codegen id : 10] Input [1]: [web_site_sk#18] (38) BroadcastExchange Input [1]: [web_site_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 8] +(40) Project [codegen id : 11] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -(41) HashAggregate [codegen id : 8] +(41) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 8] +(42) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 8] +(43) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21, count(ws_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) RowToColumnar +(44) Exchange Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) CometColumnarExchange -Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(46) ColumnarToRow [codegen id : 9] -Input [3]: [sum#22, sum#23, count#25] - -(47) HashAggregate [codegen id : 9] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index cb36bcd9a0..34ddde7687 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -1,70 +1,74 @@ -WholeStageCodegen (9) +WholeStageCodegen (12) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (3) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometSort [ws_order_number] - CometColumnarExchange [ws_order_number] #2 + InputAdapter + Exchange #1 + WholeStageCodegen (11) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometSort [ws_order_number] - CometColumnarExchange [ws_order_number] #3 + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter CometProject [ws_warehouse_sk,ws_order_number] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometSort [wr_order_number] - CometColumnarExchange [wr_order_number] #4 + InputAdapter + WholeStageCodegen (7) + Sort [wr_order_number] + InputAdapter + Exchange [wr_order_number] #4 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter CometProject [wr_order_number] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #5 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (9) ColumnarToRow InputAdapter - CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometProject [web_site_sk] + CometFilter [web_company_name,web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 45fe92329b..d8686b2b6c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -1,67 +1,62 @@ == Physical Plan == -* HashAggregate (63) -+- * ColumnarToRow (62) - +- CometColumnarExchange (61) - +- RowToColumnar (60) - +- * HashAggregate (59) - +- * HashAggregate (58) - +- * HashAggregate (57) - +- * Project (56) - +- * BroadcastHashJoin Inner BuildRight (55) - :- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * SortMergeJoin LeftSemi (35) - : : : :- * SortMergeJoin LeftSemi (18) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometSort (5) - : : : : : +- CometColumnarExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- * Project (17) - : : : : +- * SortMergeJoin Inner (16) - : : : : :- * ColumnarToRow (12) - : : : : : +- CometSort (11) - : : : : : +- CometColumnarExchange (10) - : : : : : +- CometProject (9) - : : : : : +- CometFilter (8) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) - : : : : +- * ColumnarToRow (15) - : : : : +- CometSort (14) - : : : : +- ReusedExchange (13) - : : : +- * Project (34) - : : : +- * SortMergeJoin Inner (33) - : : : :- * ColumnarToRow (24) - : : : : +- CometSort (23) - : : : : +- CometColumnarExchange (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.web_returns (19) - : : : +- * Project (32) - : : : +- * SortMergeJoin Inner (31) - : : : :- * ColumnarToRow (27) - : : : : +- CometSort (26) - : : : : +- ReusedExchange (25) - : : : +- * ColumnarToRow (30) - : : : +- CometSort (29) - : : : +- ReusedExchange (28) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.date_dim (36) - : +- BroadcastExchange (47) - : +- * ColumnarToRow (46) - : +- CometProject (45) - : +- CometFilter (44) - : +- CometScan parquet spark_catalog.default.customer_address (43) - +- BroadcastExchange (54) - +- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.web_site (50) +* HashAggregate (58) ++- Exchange (57) + +- * HashAggregate (56) + +- * HashAggregate (55) + +- * HashAggregate (54) + +- * Project (53) + +- * BroadcastHashJoin Inner BuildRight (52) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * SortMergeJoin LeftSemi (32) + : : : :- * SortMergeJoin LeftSemi (17) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- * Project (16) + : : : : +- * SortMergeJoin Inner (15) + : : : : :- * Sort (12) + : : : : : +- Exchange (11) + : : : : : +- * ColumnarToRow (10) + : : : : : +- CometProject (9) + : : : : : +- CometFilter (8) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) + : : : : +- * Sort (14) + : : : : +- ReusedExchange (13) + : : : +- * Project (31) + : : : +- * SortMergeJoin Inner (30) + : : : :- * Sort (23) + : : : : +- Exchange (22) + : : : : +- * ColumnarToRow (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.web_returns (18) + : : : +- * Project (29) + : : : +- * SortMergeJoin Inner (28) + : : : :- * Sort (25) + : : : : +- ReusedExchange (24) + : : : +- * Sort (27) + : : : +- ReusedExchange (26) + : : +- BroadcastExchange (37) + : : +- * ColumnarToRow (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.date_dim (33) + : +- BroadcastExchange (44) + : +- * ColumnarToRow (43) + : +- CometProject (42) + : +- CometFilter (41) + : +- CometScan parquet spark_catalog.default.customer_address (40) + +- BroadcastExchange (51) + +- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.web_site (47) (unknown) Scan parquet spark_catalog.default.web_sales @@ -79,16 +74,16 @@ Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -(4) CometColumnarExchange +(4) ColumnarToRow [codegen id : 1] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) CometSort +(5) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(6) ColumnarToRow [codegen id : 1] +(6) Sort [codegen id : 2] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] +Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] @@ -105,38 +100,35 @@ Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] -(10) CometColumnarExchange +(10) ColumnarToRow [codegen id : 3] Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(11) CometSort +(11) Exchange Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] +Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(12) ColumnarToRow [codegen id : 2] +(12) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#8, ws_order_number#9] +Arguments: [ws_order_number#9 ASC NULLS FIRST], false, 0 -(13) ReusedExchange [Reuses operator id: 10] +(13) ReusedExchange [Reuses operator id: 11] Output [2]: [ws_warehouse_sk#11, ws_order_number#12] -(14) CometSort +(14) Sort [codegen id : 6] Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] +Arguments: [ws_order_number#12 ASC NULLS FIRST], false, 0 -(15) ColumnarToRow [codegen id : 3] -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(16) SortMergeJoin [codegen id : 4] +(15) SortMergeJoin [codegen id : 7] Left keys [1]: [ws_order_number#9] Right keys [1]: [ws_order_number#12] Join type: Inner Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) -(17) Project [codegen id : 4] +(16) Project [codegen id : 7] Output [1]: [ws_order_number#9] Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -(18) SortMergeJoin [codegen id : 5] +(17) SortMergeJoin [codegen id : 8] Left keys [1]: [ws_order_number#4] Right keys [1]: [ws_order_number#9] Join type: LeftSemi @@ -149,66 +141,60 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(20) CometFilter +(19) CometFilter Input [2]: [wr_order_number#13, wr_returned_date_sk#14] Condition : isnotnull(wr_order_number#13) -(21) CometProject +(20) CometProject Input [2]: [wr_order_number#13, wr_returned_date_sk#14] Arguments: [wr_order_number#13], [wr_order_number#13] -(22) CometColumnarExchange +(21) ColumnarToRow [codegen id : 9] Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometSort +(22) Exchange Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] +Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) ColumnarToRow [codegen id : 6] +(23) Sort [codegen id : 10] Input [1]: [wr_order_number#13] +Arguments: [wr_order_number#13 ASC NULLS FIRST], false, 0 -(25) ReusedExchange [Reuses operator id: 10] +(24) ReusedExchange [Reuses operator id: 11] Output [2]: [ws_warehouse_sk#8, ws_order_number#9] -(26) CometSort +(25) Sort [codegen id : 12] Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] +Arguments: [ws_order_number#9 ASC NULLS FIRST], false, 0 -(27) ColumnarToRow [codegen id : 7] -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] - -(28) ReusedExchange [Reuses operator id: 10] +(26) ReusedExchange [Reuses operator id: 11] Output [2]: [ws_warehouse_sk#11, ws_order_number#12] -(29) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(30) ColumnarToRow [codegen id : 8] +(27) Sort [codegen id : 14] Input [2]: [ws_warehouse_sk#11, ws_order_number#12] +Arguments: [ws_order_number#12 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 9] +(28) SortMergeJoin [codegen id : 15] Left keys [1]: [ws_order_number#9] Right keys [1]: [ws_order_number#12] Join type: Inner Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) -(32) Project [codegen id : 9] +(29) Project [codegen id : 15] Output [1]: [ws_order_number#9] Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -(33) SortMergeJoin [codegen id : 10] +(30) SortMergeJoin [codegen id : 16] Left keys [1]: [wr_order_number#13] Right keys [1]: [ws_order_number#9] Join type: Inner Join condition: None -(34) Project [codegen id : 10] +(31) Project [codegen id : 16] Output [1]: [wr_order_number#13] Input [2]: [wr_order_number#13, ws_order_number#9] -(35) SortMergeJoin [codegen id : 14] +(32) SortMergeJoin [codegen id : 20] Left keys [1]: [ws_order_number#4] Right keys [1]: [wr_order_number#13] Join type: LeftSemi @@ -221,28 +207,28 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [d_date_sk#15, d_date#16] Condition : (((isnotnull(d_date#16) AND (d_date#16 >= 1999-02-01)) AND (d_date#16 <= 1999-04-02)) AND isnotnull(d_date_sk#15)) -(38) CometProject +(35) CometProject Input [2]: [d_date_sk#15, d_date#16] Arguments: [d_date_sk#15], [d_date_sk#15] -(39) ColumnarToRow [codegen id : 11] +(36) ColumnarToRow [codegen id : 17] Input [1]: [d_date_sk#15] -(40) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 14] +(38) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(42) Project [codegen id : 14] +(39) Project [codegen id : 20] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#15] @@ -253,28 +239,28 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(44) CometFilter +(41) CometFilter Input [2]: [ca_address_sk#17, ca_state#18] Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = IL)) AND isnotnull(ca_address_sk#17)) -(45) CometProject +(42) CometProject Input [2]: [ca_address_sk#17, ca_state#18] Arguments: [ca_address_sk#17], [ca_address_sk#17] -(46) ColumnarToRow [codegen id : 12] +(43) ColumnarToRow [codegen id : 18] Input [1]: [ca_address_sk#17] -(47) BroadcastExchange +(44) BroadcastExchange Input [1]: [ca_address_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(48) BroadcastHashJoin [codegen id : 14] +(45) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#17] Join type: Inner Join condition: None -(49) Project [codegen id : 14] +(46) Project [codegen id : 20] Output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#17] @@ -285,63 +271,57 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(51) CometFilter +(48) CometFilter Input [2]: [web_site_sk#19, web_company_name#20] Condition : ((isnotnull(web_company_name#20) AND (web_company_name#20 = pri )) AND isnotnull(web_site_sk#19)) -(52) CometProject +(49) CometProject Input [2]: [web_site_sk#19, web_company_name#20] Arguments: [web_site_sk#19], [web_site_sk#19] -(53) ColumnarToRow [codegen id : 13] +(50) ColumnarToRow [codegen id : 19] Input [1]: [web_site_sk#19] -(54) BroadcastExchange +(51) BroadcastExchange Input [1]: [web_site_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(55) BroadcastHashJoin [codegen id : 14] +(52) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#19] Join type: Inner Join condition: None -(56) Project [codegen id : 14] +(53) Project [codegen id : 20] Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#19] -(57) HashAggregate [codegen id : 14] +(54) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22] Results [3]: [ws_order_number#4, sum#23, sum#24] -(58) HashAggregate [codegen id : 14] +(55) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#23, sum#24] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22] Results [3]: [ws_order_number#4, sum#23, sum#24] -(59) HashAggregate [codegen id : 14] +(56) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#23, sum#24] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22, count(ws_order_number#4)#25] Results [3]: [sum#23, sum#24, count#26] -(60) RowToColumnar -Input [3]: [sum#23, sum#24, count#26] - -(61) CometColumnarExchange -Input [3]: [sum#23, sum#24, count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(62) ColumnarToRow [codegen id : 15] +(57) Exchange Input [3]: [sum#23, sum#24, count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(63) HashAggregate [codegen id : 15] +(58) HashAggregate [codegen id : 21] Input [3]: [sum#23, sum#24, count#26] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 704732744e..5b699890cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -1,101 +1,102 @@ -WholeStageCodegen (15) +WholeStageCodegen (21) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (14) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (5) - SortMergeJoin [ws_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometSort [ws_order_number] - CometColumnarExchange [ws_order_number] #2 + InputAdapter + Exchange #1 + WholeStageCodegen (20) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (8) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometSort [ws_order_number] - CometColumnarExchange [ws_order_number] #3 + InputAdapter + WholeStageCodegen (7) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter CometProject [ws_warehouse_sk,ws_order_number] CometFilter [ws_order_number,ws_warehouse_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometSort [ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (10) - Project [wr_order_number] - SortMergeJoin [wr_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometSort [wr_order_number] - CometColumnarExchange [wr_order_number] #4 + InputAdapter + WholeStageCodegen (6) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (16) + Project [wr_order_number] + SortMergeJoin [wr_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (10) + Sort [wr_order_number] + InputAdapter + Exchange [wr_order_number] #4 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter CometProject [wr_order_number] CometFilter [wr_order_number] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - InputAdapter - WholeStageCodegen (9) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometSort [ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometSort [ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (15) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (12) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (14) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (12) + BroadcastExchange #5 + WholeStageCodegen (17) ColumnarToRow InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (13) + BroadcastExchange #6 + WholeStageCodegen (18) ColumnarToRow InputAdapter - CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometProject [web_site_sk] + CometFilter [web_company_name,web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt index c5815879b8..b55971ac80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt @@ -1,34 +1,32 @@ == Physical Plan == -* HashAggregate (30) -+- * ColumnarToRow (29) - +- CometColumnarExchange (28) - +- RowToColumnar (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.household_demographics (5) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometProject (14) - : +- CometFilter (13) - : +- CometScan parquet spark_catalog.default.time_dim (12) - +- BroadcastExchange (23) - +- * ColumnarToRow (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan parquet spark_catalog.default.store (19) +* HashAggregate (28) ++- Exchange (27) + +- * HashAggregate (26) + +- * Project (25) + +- * BroadcastHashJoin Inner BuildRight (24) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- BroadcastExchange (9) + : : +- * ColumnarToRow (8) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.household_demographics (5) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometProject (14) + : +- CometFilter (13) + : +- CometScan parquet spark_catalog.default.time_dim (12) + +- BroadcastExchange (23) + +- * ColumnarToRow (22) + +- CometProject (21) + +- CometFilter (20) + +- CometScan parquet spark_catalog.default.store (19) (unknown) Scan parquet spark_catalog.default.store_sales @@ -152,17 +150,11 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#12] Results [1]: [count#13] -(27) RowToColumnar +(27) Exchange Input [1]: [count#13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(28) CometColumnarExchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) ColumnarToRow [codegen id : 5] -Input [1]: [count#13] - -(30) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 5] Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt index 50ec5cdde4..d1438f48eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt @@ -1,43 +1,41 @@ WholeStageCodegen (5) HashAggregate [count] [count(1),count(1),count] - ColumnarToRow - InputAdapter - CometColumnarExchange #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + InputAdapter + Exchange #1 + WholeStageCodegen (4) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_store_name,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt index 7f3d3ee195..512037f6a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt @@ -1,30 +1,27 @@ == Physical Plan == -* ColumnarToRow (26) -+- CometHashAggregate (25) - +- CometColumnarExchange (24) - +- CometHashAggregate (23) - +- CometProject (22) - +- CometSortMergeJoin (21) - :- CometSort (10) - : +- CometHashAggregate (9) - : +- CometColumnarExchange (8) - : +- RowToColumnar (7) - : +- * HashAggregate (6) - : +- * Project (5) - : +- * BroadcastHashJoin Inner BuildRight (4) - : :- * ColumnarToRow (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (3) - +- CometSort (20) - +- CometHashAggregate (19) - +- CometColumnarExchange (18) - +- RowToColumnar (17) - +- * HashAggregate (16) - +- * Project (15) - +- * BroadcastHashJoin Inner BuildRight (14) - :- * ColumnarToRow (12) - : +- CometScan parquet spark_catalog.default.catalog_sales (11) - +- ReusedExchange (13) +* HashAggregate (23) ++- Exchange (22) + +- * HashAggregate (21) + +- * Project (20) + +- * SortMergeJoin FullOuter (19) + :- * Sort (9) + : +- * HashAggregate (8) + : +- Exchange (7) + : +- * HashAggregate (6) + : +- * Project (5) + : +- * BroadcastHashJoin Inner BuildRight (4) + : :- * ColumnarToRow (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- ReusedExchange (3) + +- * Sort (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * HashAggregate (15) + +- * Project (14) + +- * BroadcastHashJoin Inner BuildRight (13) + :- * ColumnarToRow (11) + : +- CometScan parquet spark_catalog.default.catalog_sales (10) + +- ReusedExchange (12) (unknown) Scan parquet spark_catalog.default.store_sales @@ -37,7 +34,7 @@ ReadSchema: struct (2) ColumnarToRow [codegen id : 2] Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -(3) ReusedExchange [Reuses operator id: 31] +(3) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#5] (4) BroadcastHashJoin [codegen id : 2] @@ -57,21 +54,20 @@ Functions: [] Aggregate Attributes: [] Results [2]: [ss_customer_sk#2, ss_item_sk#1] -(7) RowToColumnar +(7) Exchange Input [2]: [ss_customer_sk#2, ss_item_sk#1] +Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(8) CometColumnarExchange -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(9) CometHashAggregate +(8) HashAggregate [codegen id : 3] Input [2]: [ss_customer_sk#2, ss_item_sk#1] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] +Aggregate Attributes: [] +Results [2]: [ss_customer_sk#2 AS customer_sk#6, ss_item_sk#1 AS item_sk#7] -(10) CometSort +(9) Sort [codegen id : 3] Input [2]: [customer_sk#6, item_sk#7] -Arguments: [customer_sk#6, item_sk#7], [customer_sk#6 ASC NULLS FIRST, item_sk#7 ASC NULLS FIRST] +Arguments: [customer_sk#6 ASC NULLS FIRST, item_sk#7 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] @@ -80,103 +76,104 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#11)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 5] Input [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] -(13) ReusedExchange [Reuses operator id: 31] +(12) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#12] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 5] Output [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Input [4]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10, d_date_sk#12] -(16) HashAggregate [codegen id : 4] +(15) HashAggregate [codegen id : 5] Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Functions: [] Aggregate Attributes: [] Results [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -(17) RowToColumnar -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] - -(18) CometColumnarExchange +(16) Exchange Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Arguments: hashpartitioning(cs_bill_customer_sk#8, cs_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(cs_bill_customer_sk#8, cs_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(19) CometHashAggregate +(17) HashAggregate [codegen id : 6] Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Functions: [] +Aggregate Attributes: [] +Results [2]: [cs_bill_customer_sk#8 AS customer_sk#13, cs_item_sk#9 AS item_sk#14] -(20) CometSort +(18) Sort [codegen id : 6] Input [2]: [customer_sk#13, item_sk#14] -Arguments: [customer_sk#13, item_sk#14], [customer_sk#13 ASC NULLS FIRST, item_sk#14 ASC NULLS FIRST] +Arguments: [customer_sk#13 ASC NULLS FIRST, item_sk#14 ASC NULLS FIRST], false, 0 -(21) CometSortMergeJoin -Left output [2]: [customer_sk#6, item_sk#7] -Right output [2]: [customer_sk#13, item_sk#14] -Arguments: [customer_sk#6, item_sk#7], [customer_sk#13, item_sk#14], FullOuter +(19) SortMergeJoin [codegen id : 7] +Left keys [2]: [customer_sk#6, item_sk#7] +Right keys [2]: [customer_sk#13, item_sk#14] +Join type: FullOuter +Join condition: None -(22) CometProject +(20) Project [codegen id : 7] +Output [2]: [customer_sk#6, customer_sk#13] Input [4]: [customer_sk#6, item_sk#7, customer_sk#13, item_sk#14] -Arguments: [customer_sk#6, customer_sk#13], [customer_sk#6, customer_sk#13] -(23) CometHashAggregate +(21) HashAggregate [codegen id : 7] Input [2]: [customer_sk#6, customer_sk#13] Keys: [] Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#15, sum#16, sum#17] +Results [3]: [sum#18, sum#19, sum#20] -(24) CometColumnarExchange -Input [3]: [sum#15, sum#16, sum#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(22) Exchange +Input [3]: [sum#18, sum#19, sum#20] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(25) CometHashAggregate -Input [3]: [sum#15, sum#16, sum#17] +(23) HashAggregate [codegen id : 8] +Input [3]: [sum#18, sum#19, sum#20] Keys: [] Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)] - -(26) ColumnarToRow [codegen id : 5] -Input [3]: [store_only#18, catalog_only#19, store_and_catalog#20] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END)#21, sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#22, sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#23] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END)#21 AS store_only#24, sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#22 AS catalog_only#25, sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#23 AS store_and_catalog#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * ColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.date_dim (27) +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#21] +Output [2]: [d_date_sk#5, d_month_seq#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#21] -Condition : (((isnotnull(d_month_seq#21) AND (d_month_seq#21 >= 1200)) AND (d_month_seq#21 <= 1211)) AND isnotnull(d_date_sk#5)) +(25) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#27] +Condition : (((isnotnull(d_month_seq#27) AND (d_month_seq#27 >= 1200)) AND (d_month_seq#27 <= 1211)) AND isnotnull(d_date_sk#5)) -(29) CometProject -Input [2]: [d_date_sk#5, d_month_seq#21] +(26) CometProject +Input [2]: [d_date_sk#5, d_month_seq#27] Arguments: [d_date_sk#5], [d_date_sk#5] -(30) ColumnarToRow [codegen id : 1] +(27) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(31) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt index f4381c85e2..be9c20a560 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt @@ -1,43 +1,47 @@ -WholeStageCodegen (5) - ColumnarToRow +WholeStageCodegen (8) + HashAggregate [sum,sum,sum] [sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),store_only,catalog_only,store_and_catalog,sum,sum,sum] InputAdapter - CometHashAggregate [sum,sum,sum] - CometColumnarExchange #1 - CometHashAggregate [customer_sk,customer_sk] - CometProject [customer_sk,customer_sk] - CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - CometSort [customer_sk,item_sk] - CometHashAggregate [ss_customer_sk,ss_item_sk] - CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - CometSort [customer_sk,item_sk] - CometHashAggregate [cs_bill_customer_sk,cs_item_sk] - CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #4 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [cs_bill_customer_sk,cs_item_sk] - Project [cs_bill_customer_sk,cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + Exchange #1 + WholeStageCodegen (7) + HashAggregate [customer_sk,customer_sk] [sum,sum,sum,sum,sum,sum] + Project [customer_sk,customer_sk] + SortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] + InputAdapter + WholeStageCodegen (3) + Sort [customer_sk,item_sk] + HashAggregate [ss_customer_sk,ss_item_sk] [customer_sk,item_sk] + InputAdapter + Exchange [ss_customer_sk,ss_item_sk] #2 + WholeStageCodegen (2) + HashAggregate [ss_customer_sk,ss_item_sk] + Project [ss_item_sk,ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + WholeStageCodegen (6) + Sort [customer_sk,item_sk] + HashAggregate [cs_bill_customer_sk,cs_item_sk] [customer_sk,item_sk] + InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #4 + WholeStageCodegen (5) + HashAggregate [cs_bill_customer_sk,cs_item_sk] + Project [cs_bill_customer_sk,cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index 1d9b5d4bab..b69f690810 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -1,32 +1,26 @@ == Physical Plan == -* ColumnarToRow (28) -+- CometProject (27) - +- CometSort (26) - +- CometColumnarExchange (25) - +- RowToColumnar (24) - +- * Project (23) - +- Window (22) - +- * ColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- RowToColumnar (18) - +- * HashAggregate (17) - +- * ColumnarToRow (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +* Project (22) ++- * Sort (21) + +- Exchange (20) + +- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.store_sales @@ -72,7 +66,7 @@ Join condition: None Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 33] +(10) ReusedExchange [Reuses operator id: 27] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -92,71 +86,53 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) RowToColumnar +(14) Exchange Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] - -(17) HashAggregate [codegen id : 4] +(15) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] -(18) RowToColumnar -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] - -(19) CometColumnarExchange +(16) Exchange Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) CometSort +(17) Sort [codegen id : 5] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6], [i_class#9 ASC NULLS FIRST] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(21) ColumnarToRow [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] - -(22) Window +(18) Window Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(23) Project [codegen id : 6] +(19) Project [codegen id : 6] Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] -(24) RowToColumnar +(20) Exchange Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(25) CometColumnarExchange +(21) Sort [codegen id : 7] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], true, 0 -(26) CometSort +(22) Project [codegen id : 7] +Output [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6], [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST] - -(27) CometProject -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] - -(28) ColumnarToRow [codegen id : 7] -Input [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) (unknown) Scan parquet spark_catalog.default.date_dim @@ -166,18 +142,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(30) CometFilter +(24) CometFilter Input [2]: [d_date_sk#11, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(31) CometProject +(25) CometProject Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(32) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(33) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index 7f44f1fdd6..9eabb9977c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -1,50 +1,44 @@ WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_category,i_class,i_item_id,i_item_desc,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - RowToColumnar - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometSort [i_class] - CometColumnarExchange [i_class] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] + InputAdapter + Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (6) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (5) + Sort [i_class] + InputAdapter + Exchange [i_class] #2 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt index a6e3b69c51..8420e644cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt @@ -1,38 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * HashAggregate (33) - +- * ColumnarToRow (32) - +- CometColumnarExchange (31) - +- RowToColumnar (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.ship_mode (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.call_center (16) - +- BroadcastExchange (26) - +- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +TakeOrderedAndProject (32) ++- * HashAggregate (31) + +- Exchange (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.warehouse (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.ship_mode (10) + : +- BroadcastExchange (19) + : +- * ColumnarToRow (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.call_center (16) + +- BroadcastExchange (26) + +- * ColumnarToRow (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -172,24 +170,18 @@ Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) < Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] Results [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -(30) RowToColumnar +(30) Exchange Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(31) CometColumnarExchange -Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(32) ColumnarToRow [codegen id : 6] -Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] - -(33) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 6] Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [3]: [_groupingexpression#14, sm_type#9, cc_name#11] Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] Aggregate Attributes [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] -(34) TakeOrderedAndProject +(32) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt index fc410d94ea..c5f25f0795 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt @@ -1,50 +1,48 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] WholeStageCodegen (6) HashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,sm_type,cc_name] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,w_warehouse_name] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + Exchange [_groupingexpression,sm_type,cc_name] #1 + WholeStageCodegen (5) + HashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,w_warehouse_name] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + Project [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [cc_call_center_sk] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometFilter [sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometFilter [cc_call_center_sk] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index b5d11484a8..ea5dac96e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -1,47 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * ColumnarToRow (41) - +- CometColumnarExchange (40) - +- RowToColumnar (39) - +- * HashAggregate (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * ColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (35) - +- * ColumnarToRow (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.customer_demographics (32) +TakeOrderedAndProject (41) ++- * HashAggregate (40) + +- Exchange (39) + +- * HashAggregate (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : +- BroadcastExchange (9) + : : : +- * Project (8) + : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : :- * ColumnarToRow (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (6) + : : +- BroadcastExchange (22) + : : +- Union (21) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * ColumnarToRow (17) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (18) + : +- BroadcastExchange (29) + : +- * ColumnarToRow (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.customer_address (25) + +- BroadcastExchange (35) + +- * ColumnarToRow (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.customer_demographics (32) (unknown) Scan parquet spark_catalog.default.customer @@ -68,7 +66,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) ReusedExchange [Reuses operator id: 48] +(6) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#7] (7) BroadcastHashJoin [codegen id : 2] @@ -101,7 +99,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -(13) ReusedExchange [Reuses operator id: 48] +(13) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#11] (14) BroadcastHashJoin [codegen id : 4] @@ -124,7 +122,7 @@ ReadSchema: struct (17) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(18) ReusedExchange [Reuses operator id: 48] +(18) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#16] (19) BroadcastHashJoin [codegen id : 6] @@ -220,35 +218,29 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -(39) RowToColumnar +(39) Exchange Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] +Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(40) CometColumnarExchange -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) ColumnarToRow [codegen id : 10] -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] - -(42) HashAggregate [codegen id : 10] +(40) HashAggregate [codegen id : 10] Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#31] Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] -(43) TakeOrderedAndProject +(41) TakeOrderedAndProject Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) (unknown) Scan parquet spark_catalog.default.date_dim @@ -258,18 +250,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter +(43) CometFilter Input [3]: [d_date_sk#7, d_year#38, d_moy#39] Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 4)) AND (d_moy#39 <= 7)) AND isnotnull(d_date_sk#7)) -(46) CometProject +(44) CometProject Input [3]: [d_date_sk#7, d_year#38, d_moy#39] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(48) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index c455144c22..3eb2210a6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -1,74 +1,72 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] WholeStageCodegen (10) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + BroadcastExchange #4 + Union + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) + BroadcastExchange #5 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [ca_address_sk] + CometFilter [ca_county,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index 1428a3ae34..befc87707c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -1,83 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * ColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- RowToColumnar (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * ColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- RowToColumnar (32) - : : +- * HashAggregate (31) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Project (27) - : : : +- * BroadcastHashJoin Inner BuildRight (26) - : : : :- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.store_sales (22) - : : +- ReusedExchange (28) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * ColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- RowToColumnar (51) - : +- * HashAggregate (50) - : +- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (46) - : : +- * BroadcastHashJoin Inner BuildRight (45) - : : :- * ColumnarToRow (40) - : : : +- CometFilter (39) - : : : +- CometScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (44) - : : +- * ColumnarToRow (43) - : : +- CometFilter (42) - : : +- CometScan parquet spark_catalog.default.web_sales (41) - : +- ReusedExchange (47) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * ColumnarToRow (74) - +- CometColumnarExchange (73) - +- RowToColumnar (72) - +- * HashAggregate (71) - +- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * ColumnarToRow (61) - : : +- CometFilter (60) - : : +- CometScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (65) - : +- * ColumnarToRow (64) - : +- CometFilter (63) - : +- CometScan parquet spark_catalog.default.web_sales (62) - +- ReusedExchange (68) +TakeOrderedAndProject (71) ++- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (25) + : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- BroadcastExchange (23) + : : : +- * ColumnarToRow (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- ReusedExchange (26) + : +- BroadcastExchange (50) + : +- * Filter (49) + : +- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * ColumnarToRow (36) + : : : +- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (43) + +- BroadcastExchange (68) + +- * HashAggregate (67) + +- Exchange (66) + +- * HashAggregate (65) + +- * Project (64) + +- * BroadcastHashJoin Inner BuildRight (63) + :- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * ColumnarToRow (55) + : : +- CometFilter (54) + : : +- CometScan parquet spark_catalog.default.customer (53) + : +- BroadcastExchange (59) + : +- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (62) (unknown) Scan parquet spark_catalog.default.customer @@ -123,7 +115,7 @@ Join condition: None Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(10) ReusedExchange [Reuses operator id: 83] +(10) ReusedExchange [Reuses operator id: 75] Output [2]: [d_date_sk#14, d_year#15] (11) BroadcastHashJoin [codegen id : 3] @@ -143,24 +135,18 @@ Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discoun Aggregate Attributes [1]: [sum#16] Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -(14) RowToColumnar +(14) Exchange Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 16] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] - -(17) HashAggregate [codegen id : 16] +(15) HashAggregate [codegen id : 16] Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] -(18) Filter [codegen id : 16] +(16) Filter [codegen id : 16] Input [2]: [customer_id#19, year_total#20] Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) @@ -171,11 +157,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(20) CometFilter +(18) CometFilter Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) -(21) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 6] Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] (unknown) Scan parquet spark_catalog.default.store_sales @@ -186,69 +172,63 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter +(21) CometFilter Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Condition : isnotnull(ss_customer_sk#29) -(24) ColumnarToRow [codegen id : 4] +(22) ColumnarToRow [codegen id : 4] Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -(25) BroadcastExchange +(23) BroadcastExchange Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [c_customer_sk#21] Right keys [1]: [ss_customer_sk#29] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(25) Project [codegen id : 6] Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -(28) ReusedExchange [Reuses operator id: 87] +(26) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#34, d_year#35] -(29) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#32] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(30) Project [codegen id : 6] +(28) Project [codegen id : 6] Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] -(31) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] Aggregate Attributes [1]: [sum#36] Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -(32) RowToColumnar -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] - -(33) CometColumnarExchange -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) ColumnarToRow [codegen id : 7] +(30) Exchange Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] +Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 7] Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] Results [5]: [c_customer_id#22 AS customer_id#38, c_first_name#23 AS customer_first_name#39, c_last_name#24 AS customer_last_name#40, c_email_address#28 AS customer_email_address#41, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#42] -(36) BroadcastExchange +(32) BroadcastExchange Input [5]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(33) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#38] Join type: Inner @@ -261,11 +241,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(39) CometFilter +(35) CometFilter Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] Condition : (isnotnull(c_customer_sk#43) AND isnotnull(c_customer_id#44)) -(40) ColumnarToRow [codegen id : 10] +(36) ColumnarToRow [codegen id : 10] Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] (unknown) Scan parquet spark_catalog.default.web_sales @@ -276,79 +256,73 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(42) CometFilter +(38) CometFilter Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Condition : isnotnull(ws_bill_customer_sk#51) -(43) ColumnarToRow [codegen id : 8] +(39) ColumnarToRow [codegen id : 8] Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -(44) BroadcastExchange +(40) BroadcastExchange Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 10] +(41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#43] Right keys [1]: [ws_bill_customer_sk#51] Join type: Inner Join condition: None -(46) Project [codegen id : 10] +(42) Project [codegen id : 10] Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Input [12]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -(47) ReusedExchange [Reuses operator id: 83] +(43) ReusedExchange [Reuses operator id: 75] Output [2]: [d_date_sk#56, d_year#57] -(48) BroadcastHashJoin [codegen id : 10] +(44) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#54] Right keys [1]: [d_date_sk#56] Join type: Inner Join condition: None -(49) Project [codegen id : 10] +(45) Project [codegen id : 10] Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] Input [12]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] -(50) HashAggregate [codegen id : 10] +(46) HashAggregate [codegen id : 10] Input [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] Aggregate Attributes [1]: [sum#58] Results [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] -(51) RowToColumnar +(47) Exchange Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] +Arguments: hashpartitioning(c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(52) CometColumnarExchange -Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] -Arguments: hashpartitioning(c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) ColumnarToRow [codegen id : 11] -Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] - -(54) HashAggregate [codegen id : 11] +(48) HashAggregate [codegen id : 11] Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60] Results [2]: [c_customer_id#44 AS customer_id#61, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60,18,2) AS year_total#62] -(55) Filter [codegen id : 11] +(49) Filter [codegen id : 11] Input [2]: [customer_id#61, year_total#62] Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.00)) -(56) BroadcastExchange +(50) BroadcastExchange Input [2]: [customer_id#61, year_total#62] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#61] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(52) Project [codegen id : 16] Output [8]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62] Input [9]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, customer_id#61, year_total#62] @@ -359,11 +333,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(60) CometFilter +(54) CometFilter Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_customer_id#64)) -(61) ColumnarToRow [codegen id : 14] +(55) ColumnarToRow [codegen id : 14] Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] (unknown) Scan parquet spark_catalog.default.web_sales @@ -374,89 +348,83 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(63) CometFilter +(57) CometFilter Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] Condition : isnotnull(ws_bill_customer_sk#71) -(64) ColumnarToRow [codegen id : 12] +(58) ColumnarToRow [codegen id : 12] Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] -(65) BroadcastExchange +(59) BroadcastExchange Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(66) BroadcastHashJoin [codegen id : 14] +(60) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_customer_sk#63] Right keys [1]: [ws_bill_customer_sk#71] Join type: Inner Join condition: None -(67) Project [codegen id : 14] +(61) Project [codegen id : 14] Output [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] Input [12]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] -(68) ReusedExchange [Reuses operator id: 87] +(62) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#76, d_year#77] -(69) BroadcastHashJoin [codegen id : 14] +(63) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_sold_date_sk#74] Right keys [1]: [d_date_sk#76] Join type: Inner Join condition: None -(70) Project [codegen id : 14] +(64) Project [codegen id : 14] Output [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] Input [12]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74, d_date_sk#76, d_year#77] -(71) HashAggregate [codegen id : 14] +(65) HashAggregate [codegen id : 14] Input [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] Keys [8]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))] Aggregate Attributes [1]: [sum#78] Results [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] -(72) RowToColumnar -Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] - -(73) CometColumnarExchange -Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] -Arguments: hashpartitioning(c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) ColumnarToRow [codegen id : 15] +(66) Exchange Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] +Arguments: hashpartitioning(c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(75) HashAggregate [codegen id : 15] +(67) HashAggregate [codegen id : 15] Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] Keys [8]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))#60] Results [2]: [c_customer_id#64 AS customer_id#80, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))#60,18,2) AS year_total#81] -(76) BroadcastExchange +(68) BroadcastExchange Input [2]: [customer_id#80, year_total#81] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(69) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#80] Join type: Inner Join condition: (CASE WHEN (year_total#62 > 0.00) THEN (year_total#81 / year_total#62) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END) -(78) Project [codegen id : 16] +(70) Project [codegen id : 16] Output [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] Input [10]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62, customer_id#80, year_total#81] -(79) TakeOrderedAndProject +(71) TakeOrderedAndProject Input [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] Arguments: 100, [customer_id#38 ASC NULLS FIRST, customer_first_name#39 ASC NULLS FIRST, customer_last_name#40 ASC NULLS FIRST, customer_email_address#41 ASC NULLS FIRST], [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (83) -+- * ColumnarToRow (82) - +- CometFilter (81) - +- CometScan parquet spark_catalog.default.date_dim (80) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim @@ -466,22 +434,22 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter +(73) CometFilter Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(82) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(83) BroadcastExchange +(75) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 -BroadcastExchange (87) -+- * ColumnarToRow (86) - +- CometFilter (85) - +- CometScan parquet spark_catalog.default.date_dim (84) +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) (unknown) Scan parquet spark_catalog.default.date_dim @@ -491,19 +459,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(85) CometFilter +(77) CometFilter Input [2]: [d_date_sk#34, d_year#35] Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) -(86) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] -(87) BroadcastExchange +(79) BroadcastExchange Input [2]: [d_date_sk#34, d_year#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#33 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#33 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index 44058edfa4..0a30aba051 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -7,11 +7,41 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - RowToColumnar - WholeStageCodegen (3) + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -22,109 +52,71 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #6 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 WholeStageCodegen (11) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - RowToColumnar - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index 0cdccc9b68..f3c5c46098 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -1,28 +1,24 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * ColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- RowToColumnar (18) - +- * HashAggregate (17) - +- * ColumnarToRow (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (20) ++- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.web_sales @@ -68,7 +64,7 @@ Join condition: None Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 29] +(10) ReusedExchange [Reuses operator id: 25] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -88,57 +84,45 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) RowToColumnar +(14) Exchange Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] - -(17) HashAggregate [codegen id : 4] +(15) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16] -(18) RowToColumnar -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] - -(19) CometColumnarExchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort +(16) Exchange Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16], [i_class#9 ASC NULLS FIRST] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) ColumnarToRow [codegen id : 5] +(17) Sort [codegen id : 5] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(22) Window +(18) Window Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(23) Project [codegen id : 6] +(19) Project [codegen id : 6] Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] -(24) TakeOrderedAndProject +(20) TakeOrderedAndProject Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (25) ++- * ColumnarToRow (24) + +- CometProject (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.date_dim (21) (unknown) Scan parquet spark_catalog.default.date_dim @@ -148,18 +132,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(26) CometFilter +(22) CometFilter Input [2]: [d_date_sk#11, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(27) CometProject +(23) CometProject Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(24) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(29) BroadcastExchange +(25) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 2fb0bc9aa0..1bc2538b48 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -4,41 +4,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (5) - ColumnarToRow + Sort [i_class] InputAdapter - CometSort [i_class] - CometColumnarExchange [i_class] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 + Exchange [i_class] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 643f8b02d8..6fdb365c5b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -1,94 +1,88 @@ == Physical Plan == -TakeOrderedAndProject (90) -+- * BroadcastHashJoin Inner BuildRight (89) - :- * Filter (70) - : +- * HashAggregate (69) - : +- * ColumnarToRow (68) - : +- CometColumnarExchange (67) - : +- RowToColumnar (66) - : +- * HashAggregate (65) - : +- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * BroadcastHashJoin LeftSemi BuildRight (53) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (52) - : : : +- * Project (51) - : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : :- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (49) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) - : : : :- * ColumnarToRow (37) - : : : : +- CometHashAggregate (36) - : : : : +- CometColumnarExchange (35) - : : : : +- RowToColumnar (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (47) - : : : +- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * Project (43) - : : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : : :- * ColumnarToRow (40) - : : : : : +- CometFilter (39) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) - : : : : +- ReusedExchange (41) - : : : +- ReusedExchange (44) - : : +- BroadcastExchange (59) - : : +- * BroadcastHashJoin LeftSemi BuildRight (58) - : : :- * ColumnarToRow (56) - : : : +- CometFilter (55) - : : : +- CometScan parquet spark_catalog.default.item (54) - : : +- ReusedExchange (57) - : +- ReusedExchange (62) - +- BroadcastExchange (88) - +- * Filter (87) - +- * HashAggregate (86) - +- * ColumnarToRow (85) - +- CometColumnarExchange (84) - +- RowToColumnar (83) - +- * HashAggregate (82) - +- * Project (81) - +- * BroadcastHashJoin Inner BuildRight (80) - :- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * BroadcastHashJoin LeftSemi BuildRight (75) - : : :- * ColumnarToRow (73) - : : : +- CometFilter (72) - : : : +- CometScan parquet spark_catalog.default.store_sales (71) - : : +- ReusedExchange (74) - : +- ReusedExchange (76) - +- ReusedExchange (79) +TakeOrderedAndProject (84) ++- * BroadcastHashJoin Inner BuildRight (83) + :- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : :- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * ColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * ColumnarToRow (38) + : : : : : +- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * ColumnarToRow (54) + : : : +- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + +- BroadcastExchange (82) + +- * Filter (81) + +- * HashAggregate (80) + +- Exchange (79) + +- * HashAggregate (78) + +- * Project (77) + +- * BroadcastHashJoin Inner BuildRight (76) + :- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : :- * ColumnarToRow (69) + : : : +- CometFilter (68) + : : : +- CometScan parquet spark_catalog.default.store_sales (67) + : : +- ReusedExchange (70) + : +- ReusedExchange (72) + +- ReusedExchange (75) (unknown) Scan parquet spark_catalog.default.store_sales @@ -192,7 +186,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 125] +(22) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -229,7 +223,7 @@ Join condition: None Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 125] +(30) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 6] @@ -249,20 +243,16 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) RowToColumnar +(34) Exchange Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) CometColumnarExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(36) CometHashAggregate +(35) HashAggregate [codegen id : 10] Input [3]: [brand_id#26, class_id#27, category_id#28] Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] - -(37) ColumnarToRow [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] (unknown) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] @@ -272,68 +262,68 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(39) CometFilter +(37) CometFilter Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] Condition : isnotnull(ws_item_sk#29) -(40) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 9] Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -(41) ReusedExchange [Reuses operator id: 19] +(39) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(42) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_item_sk#29] Right keys [1]: [i_item_sk#32] Join type: Inner Join condition: None -(43) Project [codegen id : 9] +(41) Project [codegen id : 9] Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(44) ReusedExchange [Reuses operator id: 125] +(42) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#36] -(45) BroadcastHashJoin [codegen id : 9] +(43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#30] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(46) Project [codegen id : 9] +(44) Project [codegen id : 9] Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] -(47) BroadcastExchange +(45) BroadcastExchange Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(48) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 10] Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] Join type: LeftSemi Join condition: None -(49) BroadcastExchange +(47) BroadcastExchange Input [3]: [brand_id#26, class_id#27, category_id#28] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(50) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#26, class_id#27, category_id#28] Join type: Inner Join condition: None -(51) Project [codegen id : 11] +(49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#37] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [ss_item_sk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(53) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi @@ -346,74 +336,68 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) -(56) ColumnarToRow [codegen id : 23] +(54) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(57) ReusedExchange [Reuses operator id: 52] +(55) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#37] -(58) BroadcastHashJoin [codegen id : 23] +(56) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#38] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(59) BroadcastExchange +(57) BroadcastExchange Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(61) Project [codegen id : 25] +(59) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(62) ReusedExchange [Reuses operator id: 116] +(60) ReusedExchange [Reuses operator id: 108] Output [1]: [d_date_sk#42] -(63) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#42] Join type: Inner Join condition: None -(64) Project [codegen id : 25] +(62) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(65) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(66) RowToColumnar -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] - -(67) CometColumnarExchange +(64) Exchange Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(68) ColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] - -(69) HashAggregate [codegen id : 52] +(65) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(70) Filter [codegen id : 52] +(66) Filter [codegen id : 52] Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) @@ -425,114 +409,106 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(72) CometFilter +(68) CometFilter Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] Condition : isnotnull(ss_item_sk#56) -(73) ColumnarToRow [codegen id : 50] +(69) ColumnarToRow [codegen id : 50] Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] -(74) ReusedExchange [Reuses operator id: 52] +(70) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#37] -(75) BroadcastHashJoin [codegen id : 50] +(71) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#56] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(76) ReusedExchange [Reuses operator id: 59] +(72) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -(77) BroadcastHashJoin [codegen id : 50] +(73) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#56] Right keys [1]: [i_item_sk#61] Join type: Inner Join condition: None -(78) Project [codegen id : 50] +(74) Project [codegen id : 50] Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -(79) ReusedExchange [Reuses operator id: 130] +(75) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#65] -(80) BroadcastHashJoin [codegen id : 50] +(76) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_sold_date_sk#59] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(81) Project [codegen id : 50] +(77) Project [codegen id : 50] Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -(82) HashAggregate [codegen id : 50] +(78) HashAggregate [codegen id : 50] Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -(83) RowToColumnar -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] - -(84) CometColumnarExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(85) ColumnarToRow [codegen id : 51] +(79) Exchange Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(86) HashAggregate [codegen id : 51] +(80) HashAggregate [codegen id : 51] Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72, count(1)#73] Results [6]: [store AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] -(87) Filter [codegen id : 51] +(81) Filter [codegen id : 51] Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(88) BroadcastExchange +(82) BroadcastExchange Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(89) BroadcastHashJoin [codegen id : 52] +(83) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Right keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Join type: Inner Join condition: None -(90) TakeOrderedAndProject +(84) TakeOrderedAndProject Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] ===== Subqueries ===== -Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (111) -+- * ColumnarToRow (110) - +- CometColumnarExchange (109) - +- RowToColumnar (108) - +- * HashAggregate (107) - +- Union (106) - :- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * ColumnarToRow (92) - : : +- CometScan parquet spark_catalog.default.store_sales (91) - : +- ReusedExchange (93) - :- * Project (100) - : +- * BroadcastHashJoin Inner BuildRight (99) - : :- * ColumnarToRow (97) - : : +- CometScan parquet spark_catalog.default.catalog_sales (96) - : +- ReusedExchange (98) - +- * Project (105) - +- * BroadcastHashJoin Inner BuildRight (104) - :- * ColumnarToRow (102) - : +- CometScan parquet spark_catalog.default.web_sales (101) - +- ReusedExchange (103) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (103) ++- Exchange (102) + +- * HashAggregate (101) + +- Union (100) + :- * Project (89) + : +- * BroadcastHashJoin Inner BuildRight (88) + : :- * ColumnarToRow (86) + : : +- CometScan parquet spark_catalog.default.store_sales (85) + : +- ReusedExchange (87) + :- * Project (94) + : +- * BroadcastHashJoin Inner BuildRight (93) + : :- * ColumnarToRow (91) + : : +- CometScan parquet spark_catalog.default.catalog_sales (90) + : +- ReusedExchange (92) + +- * Project (99) + +- * BroadcastHashJoin Inner BuildRight (98) + :- * ColumnarToRow (96) + : +- CometScan parquet spark_catalog.default.web_sales (95) + +- ReusedExchange (97) (unknown) Scan parquet spark_catalog.default.store_sales @@ -542,19 +518,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#79), dynamicpruningexpression(ss_sold_date_sk#79 IN dynamicpruning#80)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 2] +(86) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] -(93) ReusedExchange [Reuses operator id: 125] +(87) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#81] -(94) BroadcastHashJoin [codegen id : 2] +(88) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#79] Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(95) Project [codegen id : 2] +(89) Project [codegen id : 2] Output [2]: [ss_quantity#77 AS quantity#82, ss_list_price#78 AS list_price#83] Input [4]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79, d_date_sk#81] @@ -565,19 +541,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#86), dynamicpruningexpression(cs_sold_date_sk#86 IN dynamicpruning#87)] ReadSchema: struct -(97) ColumnarToRow [codegen id : 4] +(91) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] -(98) ReusedExchange [Reuses operator id: 125] +(92) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#88] -(99) BroadcastHashJoin [codegen id : 4] +(93) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#86] Right keys [1]: [d_date_sk#88] Join type: Inner Join condition: None -(100) Project [codegen id : 4] +(94) Project [codegen id : 4] Output [2]: [cs_quantity#84 AS quantity#89, cs_list_price#85 AS list_price#90] Input [4]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86, d_date_sk#88] @@ -588,60 +564,54 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 6] +(96) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] -(103) ReusedExchange [Reuses operator id: 125] +(97) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#95] -(104) BroadcastHashJoin [codegen id : 6] +(98) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#93] Right keys [1]: [d_date_sk#95] Join type: Inner Join condition: None -(105) Project [codegen id : 6] +(99) Project [codegen id : 6] Output [2]: [ws_quantity#91 AS quantity#96, ws_list_price#92 AS list_price#97] Input [4]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93, d_date_sk#95] -(106) Union +(100) Union -(107) HashAggregate [codegen id : 7] +(101) HashAggregate [codegen id : 7] Input [2]: [quantity#82, list_price#83] Keys: [] Functions [1]: [partial_avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] Aggregate Attributes [2]: [sum#98, count#99] Results [2]: [sum#100, count#101] -(108) RowToColumnar -Input [2]: [sum#100, count#101] - -(109) CometColumnarExchange -Input [2]: [sum#100, count#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(110) ColumnarToRow [codegen id : 8] +(102) Exchange Input [2]: [sum#100, count#101] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(111) HashAggregate [codegen id : 8] +(103) HashAggregate [codegen id : 8] Input [2]: [sum#100, count#101] Keys: [] Functions [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] Aggregate Attributes [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102] Results [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102 AS average_sales#103] -Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#79 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#79 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (116) -+- * ColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan parquet spark_catalog.default.date_dim (112) +BroadcastExchange (108) ++- * ColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) (unknown) Scan parquet spark_catalog.default.date_dim @@ -651,26 +621,26 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(113) CometFilter +(105) CometFilter Input [2]: [d_date_sk#42, d_week_seq#104] Condition : ((isnotnull(d_week_seq#104) AND (d_week_seq#104 = Subquery scalar-subquery#105, [id=#106])) AND isnotnull(d_date_sk#42)) -(114) CometProject +(106) CometProject Input [2]: [d_date_sk#42, d_week_seq#104] Arguments: [d_date_sk#42], [d_date_sk#42] -(115) ColumnarToRow [codegen id : 1] +(107) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#42] -(116) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 113 Hosting Expression = Subquery scalar-subquery#105, [id=#106] -* ColumnarToRow (120) -+- CometProject (119) - +- CometFilter (118) - +- CometScan parquet spark_catalog.default.date_dim (117) +Subquery:6 Hosting operator id = 105 Hosting Expression = Subquery scalar-subquery#105, [id=#106] +* ColumnarToRow (112) ++- CometProject (111) + +- CometFilter (110) + +- CometScan parquet spark_catalog.default.date_dim (109) (unknown) Scan parquet spark_catalog.default.date_dim @@ -680,23 +650,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(118) CometFilter +(110) CometFilter Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] Condition : (((((isnotnull(d_year#108) AND isnotnull(d_moy#109)) AND isnotnull(d_dom#110)) AND (d_year#108 = 1999)) AND (d_moy#109 = 12)) AND (d_dom#110 = 16)) -(119) CometProject +(111) CometProject Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] Arguments: [d_week_seq#107], [d_week_seq#107] -(120) ColumnarToRow [codegen id : 1] +(112) ColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#107] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (125) -+- * ColumnarToRow (124) - +- CometProject (123) - +- CometFilter (122) - +- CometScan parquet spark_catalog.default.date_dim (121) +BroadcastExchange (117) ++- * ColumnarToRow (116) + +- CometProject (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) (unknown) Scan parquet spark_catalog.default.date_dim @@ -706,33 +676,33 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(122) CometFilter +(114) CometFilter Input [2]: [d_date_sk#25, d_year#111] Condition : (((isnotnull(d_year#111) AND (d_year#111 >= 1998)) AND (d_year#111 <= 2000)) AND isnotnull(d_date_sk#25)) -(123) CometProject +(115) CometProject Input [2]: [d_date_sk#25, d_year#111] Arguments: [d_date_sk#25], [d_date_sk#25] -(124) ColumnarToRow [codegen id : 1] +(116) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(125) BroadcastExchange +(117) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 -BroadcastExchange (130) -+- * ColumnarToRow (129) - +- CometProject (128) - +- CometFilter (127) - +- CometScan parquet spark_catalog.default.date_dim (126) +Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 +BroadcastExchange (122) ++- * ColumnarToRow (121) + +- CometProject (120) + +- CometFilter (119) + +- CometScan parquet spark_catalog.default.date_dim (118) (unknown) Scan parquet spark_catalog.default.date_dim @@ -742,26 +712,26 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(127) CometFilter +(119) CometFilter Input [2]: [d_date_sk#65, d_week_seq#112] Condition : ((isnotnull(d_week_seq#112) AND (d_week_seq#112 = Subquery scalar-subquery#113, [id=#114])) AND isnotnull(d_date_sk#65)) -(128) CometProject +(120) CometProject Input [2]: [d_date_sk#65, d_week_seq#112] Arguments: [d_date_sk#65], [d_date_sk#65] -(129) ColumnarToRow [codegen id : 1] +(121) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#65] -(130) BroadcastExchange +(122) BroadcastExchange Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:12 Hosting operator id = 127 Hosting Expression = Subquery scalar-subquery#113, [id=#114] -* ColumnarToRow (134) -+- CometProject (133) - +- CometFilter (132) - +- CometScan parquet spark_catalog.default.date_dim (131) +Subquery:12 Hosting operator id = 119 Hosting Expression = Subquery scalar-subquery#113, [id=#114] +* ColumnarToRow (126) ++- CometProject (125) + +- CometFilter (124) + +- CometScan parquet spark_catalog.default.date_dim (123) (unknown) Scan parquet spark_catalog.default.date_dim @@ -771,15 +741,15 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(132) CometFilter +(124) CometFilter Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] Condition : (((((isnotnull(d_year#116) AND isnotnull(d_moy#117)) AND isnotnull(d_dom#118)) AND (d_year#116 = 1998)) AND (d_moy#117 = 12)) AND (d_dom#118 = 16)) -(133) CometProject +(125) CometProject Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] Arguments: [d_week_seq#115], [d_week_seq#115] -(134) ColumnarToRow [codegen id : 1] +(126) ColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#115] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 29d5a8d87c..09d8d9dde3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -5,206 +5,198 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Subquery #4 WholeStageCodegen (8) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #12 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - RowToColumnar - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] + InputAdapter + Exchange #12 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #5 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ColumnarToRow + InputAdapter + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] ColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #11 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 WholeStageCodegen (51) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - RowToColumnar - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] - Subquery #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (50) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + BroadcastExchange #15 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + Subquery #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + ReusedExchange [ss_item_sk] #3 InputAdapter - ReusedExchange [d_date_sk] #15 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index e042cdb878..a8db177f87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -1,153 +1,129 @@ == Physical Plan == -* ColumnarToRow (149) -+- CometTakeOrderedAndProject (148) - +- CometHashAggregate (147) - +- CometColumnarExchange (146) - +- RowToColumnar (145) - +- * HashAggregate (144) - +- Union (143) - :- * HashAggregate (110) - : +- * ColumnarToRow (109) - : +- CometColumnarExchange (108) - : +- RowToColumnar (107) - : +- * HashAggregate (106) - : +- Union (105) - : :- * Filter (70) - : : +- * HashAggregate (69) - : : +- * ColumnarToRow (68) - : : +- CometColumnarExchange (67) - : : +- RowToColumnar (66) - : : +- * HashAggregate (65) - : : +- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (53) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (52) - : : : : +- * Project (51) - : : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (49) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (48) - : : : : :- * ColumnarToRow (37) - : : : : : +- CometHashAggregate (36) - : : : : : +- CometColumnarExchange (35) - : : : : : +- RowToColumnar (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * ColumnarToRow (9) - : : : : : : : +- CometFilter (8) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * ColumnarToRow (12) - : : : : : : : +- CometFilter (11) - : : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * ColumnarToRow (15) - : : : : : : : : +- CometFilter (14) - : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (47) - : : : : +- * Project (46) - : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : :- * Project (43) - : : : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : : : :- * ColumnarToRow (40) - : : : : : : +- CometFilter (39) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (38) - : : : : : +- ReusedExchange (41) - : : : : +- ReusedExchange (44) - : : : +- BroadcastExchange (59) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (58) - : : : :- * ColumnarToRow (56) - : : : : +- CometFilter (55) - : : : : +- CometScan parquet spark_catalog.default.item (54) - : : : +- ReusedExchange (57) - : : +- ReusedExchange (62) - : :- * Filter (87) - : : +- * HashAggregate (86) - : : +- * ColumnarToRow (85) - : : +- CometColumnarExchange (84) - : : +- RowToColumnar (83) - : : +- * HashAggregate (82) - : : +- * Project (81) - : : +- * BroadcastHashJoin Inner BuildRight (80) - : : :- * Project (78) - : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (75) - : : : : :- * ColumnarToRow (73) - : : : : : +- CometFilter (72) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (71) - : : : : +- ReusedExchange (74) - : : : +- ReusedExchange (76) - : : +- ReusedExchange (79) - : +- * Filter (104) - : +- * HashAggregate (103) - : +- * ColumnarToRow (102) - : +- CometColumnarExchange (101) - : +- RowToColumnar (100) - : +- * HashAggregate (99) - : +- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * BroadcastHashJoin LeftSemi BuildRight (92) - : : : :- * ColumnarToRow (90) - : : : : +- CometFilter (89) - : : : : +- CometScan parquet spark_catalog.default.web_sales (88) - : : : +- ReusedExchange (91) - : : +- ReusedExchange (93) - : +- ReusedExchange (96) - :- * HashAggregate (118) - : +- * ColumnarToRow (117) - : +- CometColumnarExchange (116) - : +- RowToColumnar (115) - : +- * HashAggregate (114) - : +- * HashAggregate (113) - : +- * ColumnarToRow (112) - : +- ReusedExchange (111) - :- * HashAggregate (126) - : +- * ColumnarToRow (125) - : +- CometColumnarExchange (124) - : +- RowToColumnar (123) - : +- * HashAggregate (122) - : +- * HashAggregate (121) - : +- * ColumnarToRow (120) - : +- ReusedExchange (119) - :- * HashAggregate (134) - : +- * ColumnarToRow (133) - : +- CometColumnarExchange (132) - : +- RowToColumnar (131) - : +- * HashAggregate (130) - : +- * HashAggregate (129) - : +- * ColumnarToRow (128) - : +- ReusedExchange (127) - +- * HashAggregate (142) - +- * ColumnarToRow (141) - +- CometColumnarExchange (140) - +- RowToColumnar (139) - +- * HashAggregate (138) - +- * HashAggregate (137) - +- * ColumnarToRow (136) - +- ReusedExchange (135) +TakeOrderedAndProject (125) ++- * HashAggregate (124) + +- Exchange (123) + +- * HashAggregate (122) + +- Union (121) + :- * HashAggregate (100) + : +- Exchange (99) + : +- * HashAggregate (98) + : +- Union (97) + : :- * Filter (66) + : : +- * HashAggregate (65) + : : +- Exchange (64) + : : +- * HashAggregate (63) + : : +- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (51) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (50) + : : : : +- * Project (49) + : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : : +- BroadcastExchange (47) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : : :- * HashAggregate (35) + : : : : : +- Exchange (34) + : : : : : +- * HashAggregate (33) + : : : : : +- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * ColumnarToRow (9) + : : : : : : : +- CometFilter (8) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : : +- BroadcastExchange (27) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : : :- * ColumnarToRow (12) + : : : : : : : +- CometFilter (11) + : : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : : +- BroadcastExchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * ColumnarToRow (15) + : : : : : : : : +- CometFilter (14) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * ColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : : +- ReusedExchange (22) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (45) + : : : : +- * Project (44) + : : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : : :- * Project (41) + : : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : : :- * ColumnarToRow (38) + : : : : : : +- CometFilter (37) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : : +- ReusedExchange (39) + : : : : +- ReusedExchange (42) + : : : +- BroadcastExchange (57) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : : :- * ColumnarToRow (54) + : : : : +- CometFilter (53) + : : : : +- CometScan parquet spark_catalog.default.item (52) + : : : +- ReusedExchange (55) + : : +- ReusedExchange (60) + : :- * Filter (81) + : : +- * HashAggregate (80) + : : +- Exchange (79) + : : +- * HashAggregate (78) + : : +- * Project (77) + : : +- * BroadcastHashJoin Inner BuildRight (76) + : : :- * Project (74) + : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : : : :- * ColumnarToRow (69) + : : : : : +- CometFilter (68) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (67) + : : : : +- ReusedExchange (70) + : : : +- ReusedExchange (72) + : : +- ReusedExchange (75) + : +- * Filter (96) + : +- * HashAggregate (95) + : +- Exchange (94) + : +- * HashAggregate (93) + : +- * Project (92) + : +- * BroadcastHashJoin Inner BuildRight (91) + : :- * Project (89) + : : +- * BroadcastHashJoin Inner BuildRight (88) + : : :- * BroadcastHashJoin LeftSemi BuildRight (86) + : : : :- * ColumnarToRow (84) + : : : : +- CometFilter (83) + : : : : +- CometScan parquet spark_catalog.default.web_sales (82) + : : : +- ReusedExchange (85) + : : +- ReusedExchange (87) + : +- ReusedExchange (90) + :- * HashAggregate (105) + : +- Exchange (104) + : +- * HashAggregate (103) + : +- * HashAggregate (102) + : +- ReusedExchange (101) + :- * HashAggregate (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- ReusedExchange (106) + :- * HashAggregate (115) + : +- Exchange (114) + : +- * HashAggregate (113) + : +- * HashAggregate (112) + : +- ReusedExchange (111) + +- * HashAggregate (120) + +- Exchange (119) + +- * HashAggregate (118) + +- * HashAggregate (117) + +- ReusedExchange (116) (unknown) Scan parquet spark_catalog.default.store_sales @@ -251,7 +227,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 185] +(22) ReusedExchange [Reuses operator id: 159] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -288,7 +264,7 @@ Join condition: None Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 185] +(30) ReusedExchange [Reuses operator id: 159] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 6] @@ -308,20 +284,16 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) RowToColumnar +(34) Exchange Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) CometColumnarExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(36) CometHashAggregate +(35) HashAggregate [codegen id : 10] Input [3]: [brand_id#26, class_id#27, category_id#28] Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] - -(37) ColumnarToRow [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] (unknown) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] @@ -331,68 +303,68 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(39) CometFilter +(37) CometFilter Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] Condition : isnotnull(ws_item_sk#29) -(40) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 9] Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -(41) ReusedExchange [Reuses operator id: 19] +(39) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(42) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_item_sk#29] Right keys [1]: [i_item_sk#32] Join type: Inner Join condition: None -(43) Project [codegen id : 9] +(41) Project [codegen id : 9] Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] -(44) ReusedExchange [Reuses operator id: 185] +(42) ReusedExchange [Reuses operator id: 159] Output [1]: [d_date_sk#36] -(45) BroadcastHashJoin [codegen id : 9] +(43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#30] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(46) Project [codegen id : 9] +(44) Project [codegen id : 9] Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] -(47) BroadcastExchange +(45) BroadcastExchange Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(48) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 10] Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] Join type: LeftSemi Join condition: None -(49) BroadcastExchange +(47) BroadcastExchange Input [3]: [brand_id#26, class_id#27, category_id#28] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(50) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#26, class_id#27, category_id#28] Join type: Inner Join condition: None -(51) Project [codegen id : 11] +(49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#37] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [ss_item_sk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(53) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi @@ -405,74 +377,68 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Condition : isnotnull(i_item_sk#38) -(56) ColumnarToRow [codegen id : 23] +(54) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(57) ReusedExchange [Reuses operator id: 52] +(55) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#37] -(58) BroadcastHashJoin [codegen id : 23] +(56) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#38] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(59) BroadcastExchange +(57) BroadcastExchange Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(61) Project [codegen id : 25] +(59) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(62) ReusedExchange [Reuses operator id: 180] +(60) ReusedExchange [Reuses operator id: 154] Output [1]: [d_date_sk#42] -(63) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#42] Join type: Inner Join condition: None -(64) Project [codegen id : 25] +(62) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(65) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(66) RowToColumnar +(64) Exchange Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(67) CometColumnarExchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(68) ColumnarToRow [codegen id : 26] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] - -(69) HashAggregate [codegen id : 26] +(65) HashAggregate [codegen id : 26] Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(70) Filter [codegen id : 26] +(66) Filter [codegen id : 26] Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) @@ -484,73 +450,67 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#59), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(72) CometFilter +(68) CometFilter Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] Condition : isnotnull(cs_item_sk#56) -(73) ColumnarToRow [codegen id : 51] +(69) ColumnarToRow [codegen id : 51] Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] -(74) ReusedExchange [Reuses operator id: 52] +(70) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#37] -(75) BroadcastHashJoin [codegen id : 51] +(71) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#56] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(76) ReusedExchange [Reuses operator id: 59] +(72) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -(77) BroadcastHashJoin [codegen id : 51] +(73) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#56] Right keys [1]: [i_item_sk#61] Join type: Inner Join condition: None -(78) Project [codegen id : 51] +(74) Project [codegen id : 51] Output [6]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] Input [8]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -(79) ReusedExchange [Reuses operator id: 180] +(75) ReusedExchange [Reuses operator id: 154] Output [1]: [d_date_sk#65] -(80) BroadcastHashJoin [codegen id : 51] +(76) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_sold_date_sk#59] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(81) Project [codegen id : 51] +(77) Project [codegen id : 51] Output [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] Input [7]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -(82) HashAggregate [codegen id : 51] +(78) HashAggregate [codegen id : 51] Input [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Functions [2]: [partial_sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), partial_count(1)] Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -(83) RowToColumnar +(79) Exchange Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(84) CometColumnarExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(85) ColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] - -(86) HashAggregate [codegen id : 52] +(80) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Functions [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#72, count(1)#73] Results [6]: [catalog AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] -(87) Filter [codegen id : 52] +(81) Filter [codegen id : 52] Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) @@ -562,302 +522,248 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(89) CometFilter +(83) CometFilter Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] Condition : isnotnull(ws_item_sk#77) -(90) ColumnarToRow [codegen id : 77] +(84) ColumnarToRow [codegen id : 77] Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] -(91) ReusedExchange [Reuses operator id: 52] +(85) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#37] -(92) BroadcastHashJoin [codegen id : 77] +(86) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#77] Right keys [1]: [ss_item_sk#37] Join type: LeftSemi Join condition: None -(93) ReusedExchange [Reuses operator id: 59] +(87) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] -(94) BroadcastHashJoin [codegen id : 77] +(88) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#77] Right keys [1]: [i_item_sk#82] Join type: Inner Join condition: None -(95) Project [codegen id : 77] +(89) Project [codegen id : 77] Output [6]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#83, i_class_id#84, i_category_id#85] Input [8]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] -(96) ReusedExchange [Reuses operator id: 180] +(90) ReusedExchange [Reuses operator id: 154] Output [1]: [d_date_sk#86] -(97) BroadcastHashJoin [codegen id : 77] +(91) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_sold_date_sk#80] Right keys [1]: [d_date_sk#86] Join type: Inner Join condition: None -(98) Project [codegen id : 77] +(92) Project [codegen id : 77] Output [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] Input [7]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#83, i_class_id#84, i_category_id#85, d_date_sk#86] -(99) HashAggregate [codegen id : 77] +(93) HashAggregate [codegen id : 77] Input [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] Functions [2]: [partial_sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), partial_count(1)] Aggregate Attributes [3]: [sum#87, isEmpty#88, count#89] Results [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] -(100) RowToColumnar -Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] - -(101) CometColumnarExchange +(94) Exchange Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] -Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(102) ColumnarToRow [codegen id : 78] -Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] - -(103) HashAggregate [codegen id : 78] +(95) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] Functions [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#93, count(1)#94] Results [6]: [web AS channel#95, i_brand_id#83, i_class_id#84, i_category_id#85, sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#93 AS sales#96, count(1)#94 AS number_sales#97] -(104) Filter [codegen id : 78] +(96) Filter [codegen id : 78] Input [6]: [channel#95, i_brand_id#83, i_class_id#84, i_category_id#85, sales#96, number_sales#97] Condition : (isnotnull(sales#96) AND (cast(sales#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(105) Union +(97) Union -(106) HashAggregate [codegen id : 79] +(98) HashAggregate [codegen id : 79] Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] Aggregate Attributes [3]: [sum#98, isEmpty#99, sum#100] Results [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -(107) RowToColumnar -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(108) CometColumnarExchange +(99) Exchange Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(109) ColumnarToRow [codegen id : 80] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(110) HashAggregate [codegen id : 80] +(100) HashAggregate [codegen id : 80] Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum(sales#52), sum(number_sales#53)] Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] -(111) ReusedExchange [Reuses operator id: 108] +(101) ReusedExchange [Reuses operator id: 99] Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -(112) ColumnarToRow [codegen id : 160] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(113) HashAggregate [codegen id : 160] +(102) HashAggregate [codegen id : 160] Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum(sales#52), sum(number_sales#53)] Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] Results [5]: [channel#51, i_brand_id#39, i_class_id#40, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] -(114) HashAggregate [codegen id : 160] +(103) HashAggregate [codegen id : 160] Input [5]: [channel#51, i_brand_id#39, i_class_id#40, sum_sales#106, number_sales#107] Keys [3]: [channel#51, i_brand_id#39, i_class_id#40] Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] Results [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] -(115) RowToColumnar -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] - -(116) CometColumnarExchange -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(117) ColumnarToRow [codegen id : 161] +(104) Exchange Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(118) HashAggregate [codegen id : 161] +(105) HashAggregate [codegen id : 161] Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] Keys [3]: [channel#51, i_brand_id#39, i_class_id#40] Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] Aggregate Attributes [2]: [sum(sum_sales#106)#114, sum(number_sales#107)#115] Results [6]: [channel#51, i_brand_id#39, i_class_id#40, null AS i_category_id#116, sum(sum_sales#106)#114 AS sum(sum_sales)#117, sum(number_sales#107)#115 AS sum(number_sales)#118] -(119) ReusedExchange [Reuses operator id: 108] +(106) ReusedExchange [Reuses operator id: 99] Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -(120) ColumnarToRow [codegen id : 241] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(121) HashAggregate [codegen id : 241] +(107) HashAggregate [codegen id : 241] Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum(sales#52), sum(number_sales#53)] Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] Results [4]: [channel#51, i_brand_id#39, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] -(122) HashAggregate [codegen id : 241] +(108) HashAggregate [codegen id : 241] Input [4]: [channel#51, i_brand_id#39, sum_sales#106, number_sales#107] Keys [2]: [channel#51, i_brand_id#39] Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] Aggregate Attributes [3]: [sum#119, isEmpty#120, sum#121] Results [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] -(123) RowToColumnar -Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] - -(124) CometColumnarExchange -Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] -Arguments: hashpartitioning(channel#51, i_brand_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(125) ColumnarToRow [codegen id : 242] +(109) Exchange Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] +Arguments: hashpartitioning(channel#51, i_brand_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(126) HashAggregate [codegen id : 242] +(110) HashAggregate [codegen id : 242] Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] Keys [2]: [channel#51, i_brand_id#39] Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] Aggregate Attributes [2]: [sum(sum_sales#106)#125, sum(number_sales#107)#126] Results [6]: [channel#51, i_brand_id#39, null AS i_class_id#127, null AS i_category_id#128, sum(sum_sales#106)#125 AS sum(sum_sales)#129, sum(number_sales#107)#126 AS sum(number_sales)#130] -(127) ReusedExchange [Reuses operator id: 108] +(111) ReusedExchange [Reuses operator id: 99] Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -(128) ColumnarToRow [codegen id : 322] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(129) HashAggregate [codegen id : 322] +(112) HashAggregate [codegen id : 322] Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum(sales#52), sum(number_sales#53)] Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] Results [3]: [channel#51, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] -(130) HashAggregate [codegen id : 322] +(113) HashAggregate [codegen id : 322] Input [3]: [channel#51, sum_sales#106, number_sales#107] Keys [1]: [channel#51] Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] Aggregate Attributes [3]: [sum#131, isEmpty#132, sum#133] Results [4]: [channel#51, sum#134, isEmpty#135, sum#136] -(131) RowToColumnar -Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] - -(132) CometColumnarExchange +(114) Exchange Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] -Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(133) ColumnarToRow [codegen id : 323] -Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] - -(134) HashAggregate [codegen id : 323] +(115) HashAggregate [codegen id : 323] Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] Keys [1]: [channel#51] Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] Aggregate Attributes [2]: [sum(sum_sales#106)#137, sum(number_sales#107)#138] Results [6]: [channel#51, null AS i_brand_id#139, null AS i_class_id#140, null AS i_category_id#141, sum(sum_sales#106)#137 AS sum(sum_sales)#142, sum(number_sales#107)#138 AS sum(number_sales)#143] -(135) ReusedExchange [Reuses operator id: 108] +(116) ReusedExchange [Reuses operator id: 99] Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -(136) ColumnarToRow [codegen id : 403] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(137) HashAggregate [codegen id : 403] +(117) HashAggregate [codegen id : 403] Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum(sales#52), sum(number_sales#53)] Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] Results [2]: [sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] -(138) HashAggregate [codegen id : 403] +(118) HashAggregate [codegen id : 403] Input [2]: [sum_sales#106, number_sales#107] Keys: [] Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] Aggregate Attributes [3]: [sum#144, isEmpty#145, sum#146] Results [3]: [sum#147, isEmpty#148, sum#149] -(139) RowToColumnar +(119) Exchange Input [3]: [sum#147, isEmpty#148, sum#149] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] -(140) CometColumnarExchange -Input [3]: [sum#147, isEmpty#148, sum#149] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(141) ColumnarToRow [codegen id : 404] -Input [3]: [sum#147, isEmpty#148, sum#149] - -(142) HashAggregate [codegen id : 404] +(120) HashAggregate [codegen id : 404] Input [3]: [sum#147, isEmpty#148, sum#149] Keys: [] Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] Aggregate Attributes [2]: [sum(sum_sales#106)#150, sum(number_sales#107)#151] Results [6]: [null AS channel#152, null AS i_brand_id#153, null AS i_class_id#154, null AS i_category_id#155, sum(sum_sales#106)#150 AS sum(sum_sales)#156, sum(number_sales#107)#151 AS sum(number_sales)#157] -(143) Union +(121) Union -(144) HashAggregate [codegen id : 405] +(122) HashAggregate [codegen id : 405] Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] Functions: [] Aggregate Attributes: [] Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -(145) RowToColumnar +(123) Exchange Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107, 5), ENSURE_REQUIREMENTS, [plan_id=17] -(146) CometColumnarExchange -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] - -(147) CometHashAggregate +(124) HashAggregate [codegen id : 406] Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] Functions: [] +Aggregate Attributes: [] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -(148) CometTakeOrderedAndProject -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#51 ASC NULLS FIRST,i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#51,i_brand_id#39,i_class_id#40,i_category_id#41,sum_sales#106,number_sales#107]), 100, [channel#51 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] - -(149) ColumnarToRow [codegen id : 406] +(125) TakeOrderedAndProject Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +Arguments: 100, [channel#51 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] ===== Subqueries ===== -Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (170) -+- * ColumnarToRow (169) - +- CometColumnarExchange (168) - +- RowToColumnar (167) - +- * HashAggregate (166) - +- Union (165) - :- * Project (154) - : +- * BroadcastHashJoin Inner BuildRight (153) - : :- * ColumnarToRow (151) - : : +- CometScan parquet spark_catalog.default.store_sales (150) - : +- ReusedExchange (152) - :- * Project (159) - : +- * BroadcastHashJoin Inner BuildRight (158) - : :- * ColumnarToRow (156) - : : +- CometScan parquet spark_catalog.default.catalog_sales (155) - : +- ReusedExchange (157) - +- * Project (164) - +- * BroadcastHashJoin Inner BuildRight (163) - :- * ColumnarToRow (161) - : +- CometScan parquet spark_catalog.default.web_sales (160) - +- ReusedExchange (162) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (144) ++- Exchange (143) + +- * HashAggregate (142) + +- Union (141) + :- * Project (130) + : +- * BroadcastHashJoin Inner BuildRight (129) + : :- * ColumnarToRow (127) + : : +- CometScan parquet spark_catalog.default.store_sales (126) + : +- ReusedExchange (128) + :- * Project (135) + : +- * BroadcastHashJoin Inner BuildRight (134) + : :- * ColumnarToRow (132) + : : +- CometScan parquet spark_catalog.default.catalog_sales (131) + : +- ReusedExchange (133) + +- * Project (140) + +- * BroadcastHashJoin Inner BuildRight (139) + :- * ColumnarToRow (137) + : +- CometScan parquet spark_catalog.default.web_sales (136) + +- ReusedExchange (138) (unknown) Scan parquet spark_catalog.default.store_sales @@ -867,19 +773,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#160), dynamicpruningexpression(ss_sold_date_sk#160 IN dynamicpruning#161)] ReadSchema: struct -(151) ColumnarToRow [codegen id : 2] +(127) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#158, ss_list_price#159, ss_sold_date_sk#160] -(152) ReusedExchange [Reuses operator id: 185] +(128) ReusedExchange [Reuses operator id: 159] Output [1]: [d_date_sk#162] -(153) BroadcastHashJoin [codegen id : 2] +(129) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#160] Right keys [1]: [d_date_sk#162] Join type: Inner Join condition: None -(154) Project [codegen id : 2] +(130) Project [codegen id : 2] Output [2]: [ss_quantity#158 AS quantity#163, ss_list_price#159 AS list_price#164] Input [4]: [ss_quantity#158, ss_list_price#159, ss_sold_date_sk#160, d_date_sk#162] @@ -890,19 +796,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#167), dynamicpruningexpression(cs_sold_date_sk#167 IN dynamicpruning#168)] ReadSchema: struct -(156) ColumnarToRow [codegen id : 4] +(132) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#165, cs_list_price#166, cs_sold_date_sk#167] -(157) ReusedExchange [Reuses operator id: 175] +(133) ReusedExchange [Reuses operator id: 149] Output [1]: [d_date_sk#169] -(158) BroadcastHashJoin [codegen id : 4] +(134) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#167] Right keys [1]: [d_date_sk#169] Join type: Inner Join condition: None -(159) Project [codegen id : 4] +(135) Project [codegen id : 4] Output [2]: [cs_quantity#165 AS quantity#170, cs_list_price#166 AS list_price#171] Input [4]: [cs_quantity#165, cs_list_price#166, cs_sold_date_sk#167, d_date_sk#169] @@ -913,56 +819,50 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#174), dynamicpruningexpression(ws_sold_date_sk#174 IN dynamicpruning#175)] ReadSchema: struct -(161) ColumnarToRow [codegen id : 6] +(137) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#172, ws_list_price#173, ws_sold_date_sk#174] -(162) ReusedExchange [Reuses operator id: 175] +(138) ReusedExchange [Reuses operator id: 149] Output [1]: [d_date_sk#176] -(163) BroadcastHashJoin [codegen id : 6] +(139) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#174] Right keys [1]: [d_date_sk#176] Join type: Inner Join condition: None -(164) Project [codegen id : 6] +(140) Project [codegen id : 6] Output [2]: [ws_quantity#172 AS quantity#177, ws_list_price#173 AS list_price#178] Input [4]: [ws_quantity#172, ws_list_price#173, ws_sold_date_sk#174, d_date_sk#176] -(165) Union +(141) Union -(166) HashAggregate [codegen id : 7] +(142) HashAggregate [codegen id : 7] Input [2]: [quantity#163, list_price#164] Keys: [] Functions [1]: [partial_avg((cast(quantity#163 as decimal(10,0)) * list_price#164))] Aggregate Attributes [2]: [sum#179, count#180] Results [2]: [sum#181, count#182] -(167) RowToColumnar -Input [2]: [sum#181, count#182] - -(168) CometColumnarExchange -Input [2]: [sum#181, count#182] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] - -(169) ColumnarToRow [codegen id : 8] +(143) Exchange Input [2]: [sum#181, count#182] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] -(170) HashAggregate [codegen id : 8] +(144) HashAggregate [codegen id : 8] Input [2]: [sum#181, count#182] Keys: [] Functions [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))] Aggregate Attributes [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))#183] Results [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))#183 AS average_sales#184] -Subquery:2 Hosting operator id = 150 Hosting Expression = ss_sold_date_sk#160 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#160 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 155 Hosting Expression = cs_sold_date_sk#167 IN dynamicpruning#168 -BroadcastExchange (175) -+- * ColumnarToRow (174) - +- CometProject (173) - +- CometFilter (172) - +- CometScan parquet spark_catalog.default.date_dim (171) +Subquery:3 Hosting operator id = 131 Hosting Expression = cs_sold_date_sk#167 IN dynamicpruning#168 +BroadcastExchange (149) ++- * ColumnarToRow (148) + +- CometProject (147) + +- CometFilter (146) + +- CometScan parquet spark_catalog.default.date_dim (145) (unknown) Scan parquet spark_catalog.default.date_dim @@ -972,29 +872,29 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(172) CometFilter +(146) CometFilter Input [2]: [d_date_sk#169, d_year#185] Condition : (((isnotnull(d_year#185) AND (d_year#185 >= 1998)) AND (d_year#185 <= 2000)) AND isnotnull(d_date_sk#169)) -(173) CometProject +(147) CometProject Input [2]: [d_date_sk#169, d_year#185] Arguments: [d_date_sk#169], [d_date_sk#169] -(174) ColumnarToRow [codegen id : 1] +(148) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#169] -(175) BroadcastExchange +(149) BroadcastExchange Input [1]: [d_date_sk#169] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -Subquery:4 Hosting operator id = 160 Hosting Expression = ws_sold_date_sk#174 IN dynamicpruning#168 +Subquery:4 Hosting operator id = 136 Hosting Expression = ws_sold_date_sk#174 IN dynamicpruning#168 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (180) -+- * ColumnarToRow (179) - +- CometProject (178) - +- CometFilter (177) - +- CometScan parquet spark_catalog.default.date_dim (176) +BroadcastExchange (154) ++- * ColumnarToRow (153) + +- CometProject (152) + +- CometFilter (151) + +- CometScan parquet spark_catalog.default.date_dim (150) (unknown) Scan parquet spark_catalog.default.date_dim @@ -1004,27 +904,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(177) CometFilter +(151) CometFilter Input [3]: [d_date_sk#42, d_year#186, d_moy#187] Condition : ((((isnotnull(d_year#186) AND isnotnull(d_moy#187)) AND (d_year#186 = 2000)) AND (d_moy#187 = 11)) AND isnotnull(d_date_sk#42)) -(178) CometProject +(152) CometProject Input [3]: [d_date_sk#42, d_year#186, d_moy#187] Arguments: [d_date_sk#42], [d_date_sk#42] -(179) ColumnarToRow [codegen id : 1] +(153) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#42] -(180) BroadcastExchange +(154) BroadcastExchange Input [1]: [d_date_sk#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (185) -+- * ColumnarToRow (184) - +- CometProject (183) - +- CometFilter (182) - +- CometScan parquet spark_catalog.default.date_dim (181) +BroadcastExchange (159) ++- * ColumnarToRow (158) + +- CometProject (157) + +- CometFilter (156) + +- CometScan parquet spark_catalog.default.date_dim (155) (unknown) Scan parquet spark_catalog.default.date_dim @@ -1034,31 +934,31 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(182) CometFilter +(156) CometFilter Input [2]: [d_date_sk#25, d_year#188] Condition : (((isnotnull(d_year#188) AND (d_year#188 >= 1999)) AND (d_year#188 <= 2001)) AND isnotnull(d_date_sk#25)) -(183) CometProject +(157) CometProject Input [2]: [d_date_sk#25, d_year#188] Arguments: [d_date_sk#25], [d_date_sk#25] -(184) ColumnarToRow [codegen id : 1] +(158) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(185) BroadcastExchange +(159) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:10 Hosting operator id = 71 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:12 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 90bff1665b..a203f9620b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -1,287 +1,261 @@ -WholeStageCodegen (406) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - RowToColumnar - WholeStageCodegen (405) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - RowToColumnar - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (26) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #14 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + WholeStageCodegen (406) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + InputAdapter + Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + WholeStageCodegen (405) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + InputAdapter + Union + WholeStageCodegen (80) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 + WholeStageCodegen (79) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (26) + Filter [sales] + Subquery #3 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + InputAdapter + Exchange #14 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #15 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #15 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #15 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #3 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #15 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 - RowToColumnar - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - ColumnarToRow - InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #7 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + BroadcastExchange #10 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 - RowToColumnar - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 - RowToColumnar - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + BroadcastExchange #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) - HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id] #18 - RowToColumnar - WholeStageCodegen (160) - HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - ColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) - HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id] #19 - RowToColumnar - WholeStageCodegen (241) - HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - ColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) - HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel] #20 - RowToColumnar - WholeStageCodegen (322) - HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - ColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) - HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange #21 - RowToColumnar - WholeStageCodegen (403) - HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - ColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (52) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (51) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (78) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (77) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (161) + HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + Exchange [channel,i_brand_id,i_class_id] #18 + WholeStageCodegen (160) + HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (242) + HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + Exchange [channel,i_brand_id] #19 + WholeStageCodegen (241) + HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (323) + HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + Exchange [channel] #20 + WholeStageCodegen (322) + HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (404) + HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + Exchange #21 + WholeStageCodegen (403) + HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt index 70d144d28c..4d23b269c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt @@ -1,167 +1,157 @@ == Physical Plan == -TakeOrderedAndProject (163) -+- Union (162) - :- * HashAggregate (43) - : +- * ColumnarToRow (42) - : +- CometColumnarExchange (41) - : +- RowToColumnar (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * ColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan parquet spark_catalog.default.customer (11) - : : : : +- BroadcastExchange (21) - : : : : +- * ColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) - : : : +- BroadcastExchange (27) - : : : +- * ColumnarToRow (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.customer_address (24) - : : +- ReusedExchange (30) - : +- BroadcastExchange (36) - : +- * ColumnarToRow (35) - : +- CometFilter (34) - : +- CometScan parquet spark_catalog.default.item (33) - :- * HashAggregate (72) - : +- * ColumnarToRow (71) - : +- CometColumnarExchange (70) - : +- RowToColumnar (69) - : +- * HashAggregate (68) - : +- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * Project (55) - : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : :- * Project (52) - : : : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : : : :- * Project (49) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : : : :- * ColumnarToRow (46) - : : : : : : : +- CometFilter (45) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (44) - : : : : : : +- ReusedExchange (47) - : : : : : +- ReusedExchange (50) - : : : : +- ReusedExchange (53) - : : : +- BroadcastExchange (59) - : : : +- * ColumnarToRow (58) - : : : +- CometFilter (57) - : : : +- CometScan parquet spark_catalog.default.customer_address (56) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - :- * HashAggregate (102) - : +- * ColumnarToRow (101) - : +- CometColumnarExchange (100) - : +- RowToColumnar (99) - : +- * HashAggregate (98) - : +- * Project (97) - : +- * BroadcastHashJoin Inner BuildRight (96) - : :- * Project (94) - : : +- * BroadcastHashJoin Inner BuildRight (93) - : : :- * Project (91) - : : : +- * BroadcastHashJoin Inner BuildRight (90) - : : : :- * Project (84) - : : : : +- * BroadcastHashJoin Inner BuildRight (83) - : : : : :- * Project (81) - : : : : : +- * BroadcastHashJoin Inner BuildRight (80) - : : : : : :- * Project (78) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : : : : :- * ColumnarToRow (75) - : : : : : : : +- CometFilter (74) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) - : : : : : : +- ReusedExchange (76) - : : : : : +- ReusedExchange (79) - : : : : +- ReusedExchange (82) - : : : +- BroadcastExchange (89) - : : : +- * ColumnarToRow (88) - : : : +- CometProject (87) - : : : +- CometFilter (86) - : : : +- CometScan parquet spark_catalog.default.customer_address (85) - : : +- ReusedExchange (92) - : +- ReusedExchange (95) - :- * HashAggregate (132) - : +- * ColumnarToRow (131) - : +- CometColumnarExchange (130) - : +- RowToColumnar (129) - : +- * HashAggregate (128) - : +- * Project (127) - : +- * BroadcastHashJoin Inner BuildRight (126) - : :- * Project (124) - : : +- * BroadcastHashJoin Inner BuildRight (123) - : : :- * Project (121) - : : : +- * BroadcastHashJoin Inner BuildRight (120) - : : : :- * Project (114) - : : : : +- * BroadcastHashJoin Inner BuildRight (113) - : : : : :- * Project (111) - : : : : : +- * BroadcastHashJoin Inner BuildRight (110) - : : : : : :- * Project (108) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (107) - : : : : : : :- * ColumnarToRow (105) - : : : : : : : +- CometFilter (104) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (103) - : : : : : : +- ReusedExchange (106) - : : : : : +- ReusedExchange (109) - : : : : +- ReusedExchange (112) - : : : +- BroadcastExchange (119) - : : : +- * ColumnarToRow (118) - : : : +- CometProject (117) - : : : +- CometFilter (116) - : : : +- CometScan parquet spark_catalog.default.customer_address (115) - : : +- ReusedExchange (122) - : +- ReusedExchange (125) - +- * HashAggregate (161) - +- * ColumnarToRow (160) - +- CometColumnarExchange (159) - +- RowToColumnar (158) - +- * HashAggregate (157) - +- * Project (156) - +- * BroadcastHashJoin Inner BuildRight (155) - :- * Project (150) - : +- * BroadcastHashJoin Inner BuildRight (149) - : :- * Project (147) - : : +- * BroadcastHashJoin Inner BuildRight (146) - : : :- * Project (144) - : : : +- * BroadcastHashJoin Inner BuildRight (143) - : : : :- * Project (141) - : : : : +- * BroadcastHashJoin Inner BuildRight (140) - : : : : :- * Project (138) - : : : : : +- * BroadcastHashJoin Inner BuildRight (137) - : : : : : :- * ColumnarToRow (135) - : : : : : : +- CometFilter (134) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (133) - : : : : : +- ReusedExchange (136) - : : : : +- ReusedExchange (139) - : : : +- ReusedExchange (142) - : : +- ReusedExchange (145) - : +- ReusedExchange (148) - +- BroadcastExchange (154) - +- * ColumnarToRow (153) - +- CometFilter (152) - +- CometScan parquet spark_catalog.default.item (151) +TakeOrderedAndProject (153) ++- Union (152) + :- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (32) + : : +- * BroadcastHashJoin Inner BuildRight (31) + : : :- * Project (29) + : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : :- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * ColumnarToRow (7) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- CometProject (13) + : : : : : +- CometFilter (12) + : : : : : +- CometScan parquet spark_catalog.default.customer (11) + : : : : +- BroadcastExchange (21) + : : : : +- * ColumnarToRow (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) + : : : +- BroadcastExchange (27) + : : : +- * ColumnarToRow (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.customer_address (24) + : : +- ReusedExchange (30) + : +- BroadcastExchange (36) + : +- * ColumnarToRow (35) + : +- CometFilter (34) + : +- CometScan parquet spark_catalog.default.item (33) + :- * HashAggregate (68) + : +- Exchange (67) + : +- * HashAggregate (66) + : +- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * Project (53) + : : : : +- * BroadcastHashJoin Inner BuildRight (52) + : : : : :- * Project (50) + : : : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : : : :- * Project (47) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : : : :- * ColumnarToRow (44) + : : : : : : : +- CometFilter (43) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (42) + : : : : : : +- ReusedExchange (45) + : : : : : +- ReusedExchange (48) + : : : : +- ReusedExchange (51) + : : : +- BroadcastExchange (57) + : : : +- * ColumnarToRow (56) + : : : +- CometFilter (55) + : : : +- CometScan parquet spark_catalog.default.customer_address (54) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + :- * HashAggregate (96) + : +- Exchange (95) + : +- * HashAggregate (94) + : +- * Project (93) + : +- * BroadcastHashJoin Inner BuildRight (92) + : :- * Project (90) + : : +- * BroadcastHashJoin Inner BuildRight (89) + : : :- * Project (87) + : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : :- * Project (80) + : : : : +- * BroadcastHashJoin Inner BuildRight (79) + : : : : :- * Project (77) + : : : : : +- * BroadcastHashJoin Inner BuildRight (76) + : : : : : :- * Project (74) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : : : : :- * ColumnarToRow (71) + : : : : : : : +- CometFilter (70) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (69) + : : : : : : +- ReusedExchange (72) + : : : : : +- ReusedExchange (75) + : : : : +- ReusedExchange (78) + : : : +- BroadcastExchange (85) + : : : +- * ColumnarToRow (84) + : : : +- CometProject (83) + : : : +- CometFilter (82) + : : : +- CometScan parquet spark_catalog.default.customer_address (81) + : : +- ReusedExchange (88) + : +- ReusedExchange (91) + :- * HashAggregate (124) + : +- Exchange (123) + : +- * HashAggregate (122) + : +- * Project (121) + : +- * BroadcastHashJoin Inner BuildRight (120) + : :- * Project (118) + : : +- * BroadcastHashJoin Inner BuildRight (117) + : : :- * Project (115) + : : : +- * BroadcastHashJoin Inner BuildRight (114) + : : : :- * Project (108) + : : : : +- * BroadcastHashJoin Inner BuildRight (107) + : : : : :- * Project (105) + : : : : : +- * BroadcastHashJoin Inner BuildRight (104) + : : : : : :- * Project (102) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (101) + : : : : : : :- * ColumnarToRow (99) + : : : : : : : +- CometFilter (98) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (97) + : : : : : : +- ReusedExchange (100) + : : : : : +- ReusedExchange (103) + : : : : +- ReusedExchange (106) + : : : +- BroadcastExchange (113) + : : : +- * ColumnarToRow (112) + : : : +- CometProject (111) + : : : +- CometFilter (110) + : : : +- CometScan parquet spark_catalog.default.customer_address (109) + : : +- ReusedExchange (116) + : +- ReusedExchange (119) + +- * HashAggregate (151) + +- Exchange (150) + +- * HashAggregate (149) + +- * Project (148) + +- * BroadcastHashJoin Inner BuildRight (147) + :- * Project (142) + : +- * BroadcastHashJoin Inner BuildRight (141) + : :- * Project (139) + : : +- * BroadcastHashJoin Inner BuildRight (138) + : : :- * Project (136) + : : : +- * BroadcastHashJoin Inner BuildRight (135) + : : : :- * Project (133) + : : : : +- * BroadcastHashJoin Inner BuildRight (132) + : : : : :- * Project (130) + : : : : : +- * BroadcastHashJoin Inner BuildRight (129) + : : : : : :- * ColumnarToRow (127) + : : : : : : +- CometFilter (126) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (125) + : : : : : +- ReusedExchange (128) + : : : : +- ReusedExchange (131) + : : : +- ReusedExchange (134) + : : +- ReusedExchange (137) + : +- ReusedExchange (140) + +- BroadcastExchange (146) + +- * ColumnarToRow (145) + +- CometFilter (144) + +- CometScan parquet spark_catalog.default.item (143) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -299,7 +289,7 @@ Join condition: None Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -(30) ReusedExchange [Reuses operator id: 168] +(30) ReusedExchange [Reuses operator id: 158] Output [1]: [d_date_sk#25] (31) BroadcastHashJoin [codegen id : 7] @@ -347,17 +337,11 @@ Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30) Aggregate Attributes [14]: [sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48] Results [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -(40) RowToColumnar +(40) Exchange Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, ca_county#22, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) CometColumnarExchange -Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, ca_county#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) ColumnarToRow [codegen id : 8] -Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] - -(43) HashAggregate [codegen id : 8] +(41) HashAggregate [codegen id : 8] Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Keys [4]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] @@ -372,49 +356,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(45) CometFilter +(43) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(46) ColumnarToRow [codegen id : 15] +(44) ColumnarToRow [codegen id : 15] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(47) ReusedExchange [Reuses operator id: 8] +(45) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(48) BroadcastHashJoin [codegen id : 15] +(46) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(49) Project [codegen id : 15] +(47) Project [codegen id : 15] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(50) ReusedExchange [Reuses operator id: 15] +(48) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(51) BroadcastHashJoin [codegen id : 15] +(49) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(52) Project [codegen id : 15] +(50) Project [codegen id : 15] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(53) ReusedExchange [Reuses operator id: 21] +(51) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(54) BroadcastHashJoin [codegen id : 15] +(52) BroadcastHashJoin [codegen id : 15] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(55) Project [codegen id : 15] +(53) Project [codegen id : 15] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -425,71 +409,65 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(57) CometFilter +(55) CometFilter Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(58) ColumnarToRow [codegen id : 12] +(56) ColumnarToRow [codegen id : 12] Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] -(59) BroadcastExchange +(57) BroadcastExchange Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(60) BroadcastHashJoin [codegen id : 15] +(58) BroadcastHashJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(61) Project [codegen id : 15] +(59) Project [codegen id : 15] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_state#23, ca_country#24] -(62) ReusedExchange [Reuses operator id: 168] +(60) ReusedExchange [Reuses operator id: 158] Output [1]: [d_date_sk#25] -(63) BroadcastHashJoin [codegen id : 15] +(61) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(64) Project [codegen id : 15] +(62) Project [codegen id : 15] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24, d_date_sk#25] -(65) ReusedExchange [Reuses operator id: 36] +(63) ReusedExchange [Reuses operator id: 36] Output [2]: [i_item_sk#26, i_item_id#27] -(66) BroadcastHashJoin [codegen id : 15] +(64) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(67) Project [codegen id : 15] +(65) Project [codegen id : 15] Output [10]: [i_item_id#27, ca_country#24, ca_state#23, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24, i_item_sk#26, i_item_id#27] -(68) HashAggregate [codegen id : 15] +(66) HashAggregate [codegen id : 15] Input [10]: [i_item_id#27, ca_country#24, ca_state#23, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [3]: [i_item_id#27, ca_country#24, ca_state#23] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] Aggregate Attributes [14]: [sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] Results [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] -(69) RowToColumnar -Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] - -(70) CometColumnarExchange -Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] -Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(71) ColumnarToRow [codegen id : 16] +(67) Exchange Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] +Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(72) HashAggregate [codegen id : 16] +(68) HashAggregate [codegen id : 16] Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] Keys [3]: [i_item_id#27, ca_country#24, ca_state#23] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] @@ -504,49 +482,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(74) CometFilter +(70) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(75) ColumnarToRow [codegen id : 23] +(71) ColumnarToRow [codegen id : 23] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(76) ReusedExchange [Reuses operator id: 8] +(72) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(77) BroadcastHashJoin [codegen id : 23] +(73) BroadcastHashJoin [codegen id : 23] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(78) Project [codegen id : 23] +(74) Project [codegen id : 23] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(79) ReusedExchange [Reuses operator id: 15] +(75) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(80) BroadcastHashJoin [codegen id : 23] +(76) BroadcastHashJoin [codegen id : 23] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(81) Project [codegen id : 23] +(77) Project [codegen id : 23] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(82) ReusedExchange [Reuses operator id: 21] +(78) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(83) BroadcastHashJoin [codegen id : 23] +(79) BroadcastHashJoin [codegen id : 23] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(84) Project [codegen id : 23] +(80) Project [codegen id : 23] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -557,75 +535,69 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(86) CometFilter +(82) CometFilter Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(87) CometProject +(83) CometProject Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_country#24], [ca_address_sk#21, ca_country#24] -(88) ColumnarToRow [codegen id : 20] +(84) ColumnarToRow [codegen id : 20] Input [2]: [ca_address_sk#21, ca_country#24] -(89) BroadcastExchange +(85) BroadcastExchange Input [2]: [ca_address_sk#21, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(90) BroadcastHashJoin [codegen id : 23] +(86) BroadcastHashJoin [codegen id : 23] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(91) Project [codegen id : 23] +(87) Project [codegen id : 23] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_country#24] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_country#24] -(92) ReusedExchange [Reuses operator id: 168] +(88) ReusedExchange [Reuses operator id: 158] Output [1]: [d_date_sk#25] -(93) BroadcastHashJoin [codegen id : 23] +(89) BroadcastHashJoin [codegen id : 23] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(94) Project [codegen id : 23] +(90) Project [codegen id : 23] Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_country#24, d_date_sk#25] -(95) ReusedExchange [Reuses operator id: 36] +(91) ReusedExchange [Reuses operator id: 36] Output [2]: [i_item_sk#26, i_item_id#27] -(96) BroadcastHashJoin [codegen id : 23] +(92) BroadcastHashJoin [codegen id : 23] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(97) Project [codegen id : 23] +(93) Project [codegen id : 23] Output [9]: [i_item_id#27, ca_country#24, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24, i_item_sk#26, i_item_id#27] -(98) HashAggregate [codegen id : 23] +(94) HashAggregate [codegen id : 23] Input [9]: [i_item_id#27, ca_country#24, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [2]: [i_item_id#27, ca_country#24] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] Aggregate Attributes [14]: [sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] Results [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] -(99) RowToColumnar -Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] - -(100) CometColumnarExchange +(95) Exchange Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] -Arguments: hashpartitioning(i_item_id#27, ca_country#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Arguments: hashpartitioning(i_item_id#27, ca_country#24, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(101) ColumnarToRow [codegen id : 24] -Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] - -(102) HashAggregate [codegen id : 24] +(96) HashAggregate [codegen id : 24] Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] Keys [2]: [i_item_id#27, ca_country#24] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] @@ -640,49 +612,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(104) CometFilter +(98) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(105) ColumnarToRow [codegen id : 31] +(99) ColumnarToRow [codegen id : 31] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(106) ReusedExchange [Reuses operator id: 8] +(100) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(107) BroadcastHashJoin [codegen id : 31] +(101) BroadcastHashJoin [codegen id : 31] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(108) Project [codegen id : 31] +(102) Project [codegen id : 31] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(109) ReusedExchange [Reuses operator id: 15] +(103) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(110) BroadcastHashJoin [codegen id : 31] +(104) BroadcastHashJoin [codegen id : 31] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(111) Project [codegen id : 31] +(105) Project [codegen id : 31] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(112) ReusedExchange [Reuses operator id: 21] +(106) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(113) BroadcastHashJoin [codegen id : 31] +(107) BroadcastHashJoin [codegen id : 31] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(114) Project [codegen id : 31] +(108) Project [codegen id : 31] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -693,75 +665,69 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(116) CometFilter +(110) CometFilter Input [2]: [ca_address_sk#21, ca_state#23] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(117) CometProject +(111) CometProject Input [2]: [ca_address_sk#21, ca_state#23] Arguments: [ca_address_sk#21], [ca_address_sk#21] -(118) ColumnarToRow [codegen id : 28] +(112) ColumnarToRow [codegen id : 28] Input [1]: [ca_address_sk#21] -(119) BroadcastExchange +(113) BroadcastExchange Input [1]: [ca_address_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(120) BroadcastHashJoin [codegen id : 31] +(114) BroadcastHashJoin [codegen id : 31] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(121) Project [codegen id : 31] +(115) Project [codegen id : 31] Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21] -(122) ReusedExchange [Reuses operator id: 168] +(116) ReusedExchange [Reuses operator id: 158] Output [1]: [d_date_sk#25] -(123) BroadcastHashJoin [codegen id : 31] +(117) BroadcastHashJoin [codegen id : 31] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(124) Project [codegen id : 31] +(118) Project [codegen id : 31] Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, d_date_sk#25] -(125) ReusedExchange [Reuses operator id: 36] +(119) ReusedExchange [Reuses operator id: 36] Output [2]: [i_item_sk#26, i_item_id#27] -(126) BroadcastHashJoin [codegen id : 31] +(120) BroadcastHashJoin [codegen id : 31] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(127) Project [codegen id : 31] +(121) Project [codegen id : 31] Output [8]: [i_item_id#27, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_sk#26, i_item_id#27] -(128) HashAggregate [codegen id : 31] +(122) HashAggregate [codegen id : 31] Input [8]: [i_item_id#27, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [1]: [i_item_id#27] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] Aggregate Attributes [14]: [sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180] Results [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] -(129) RowToColumnar -Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] - -(130) CometColumnarExchange -Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(131) ColumnarToRow [codegen id : 32] +(123) Exchange Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] +Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(132) HashAggregate [codegen id : 32] +(124) HashAggregate [codegen id : 32] Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] Keys [1]: [i_item_id#27] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] @@ -776,75 +742,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(134) CometFilter +(126) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(135) ColumnarToRow [codegen id : 39] +(127) ColumnarToRow [codegen id : 39] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(136) ReusedExchange [Reuses operator id: 8] +(128) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(137) BroadcastHashJoin [codegen id : 39] +(129) BroadcastHashJoin [codegen id : 39] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(138) Project [codegen id : 39] +(130) Project [codegen id : 39] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(139) ReusedExchange [Reuses operator id: 15] +(131) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(140) BroadcastHashJoin [codegen id : 39] +(132) BroadcastHashJoin [codegen id : 39] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(141) Project [codegen id : 39] +(133) Project [codegen id : 39] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(142) ReusedExchange [Reuses operator id: 21] +(134) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#20] -(143) BroadcastHashJoin [codegen id : 39] +(135) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(144) Project [codegen id : 39] +(136) Project [codegen id : 39] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -(145) ReusedExchange [Reuses operator id: 119] +(137) ReusedExchange [Reuses operator id: 113] Output [1]: [ca_address_sk#21] -(146) BroadcastHashJoin [codegen id : 39] +(138) BroadcastHashJoin [codegen id : 39] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(147) Project [codegen id : 39] +(139) Project [codegen id : 39] Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21] -(148) ReusedExchange [Reuses operator id: 168] +(140) ReusedExchange [Reuses operator id: 158] Output [1]: [d_date_sk#25] -(149) BroadcastHashJoin [codegen id : 39] +(141) BroadcastHashJoin [codegen id : 39] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(150) Project [codegen id : 39] +(142) Project [codegen id : 39] Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, d_date_sk#25] @@ -855,65 +821,59 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(152) CometFilter +(144) CometFilter Input [1]: [i_item_sk#26] Condition : isnotnull(i_item_sk#26) -(153) ColumnarToRow [codegen id : 38] +(145) ColumnarToRow [codegen id : 38] Input [1]: [i_item_sk#26] -(154) BroadcastExchange +(146) BroadcastExchange Input [1]: [i_item_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(155) BroadcastHashJoin [codegen id : 39] +(147) BroadcastHashJoin [codegen id : 39] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(156) Project [codegen id : 39] +(148) Project [codegen id : 39] Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_sk#26] -(157) HashAggregate [codegen id : 39] +(149) HashAggregate [codegen id : 39] Input [7]: [agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys: [] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] Aggregate Attributes [14]: [sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224, sum#225, count#226] Results [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] -(158) RowToColumnar -Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] - -(159) CometColumnarExchange -Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(160) ColumnarToRow [codegen id : 40] +(150) Exchange Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(161) HashAggregate [codegen id : 40] +(151) HashAggregate [codegen id : 40] Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] Keys: [] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] Aggregate Attributes [7]: [avg(agg1#28)#241, avg(agg2#29)#242, avg(agg3#30)#243, avg(agg4#31)#244, avg(agg5#32)#245, avg(agg6#33)#246, avg(agg7#34)#247] Results [11]: [null AS i_item_id#248, null AS ca_country#249, null AS ca_state#250, null AS county#251, avg(agg1#28)#241 AS agg1#252, avg(agg2#29)#242 AS agg2#253, avg(agg3#30)#243 AS agg3#254, avg(agg4#31)#244 AS agg4#255, avg(agg5#32)#245 AS agg5#256, avg(agg6#33)#246 AS agg6#257, avg(agg7#34)#247 AS agg7#258] -(162) Union +(152) Union -(163) TakeOrderedAndProject +(153) TakeOrderedAndProject Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] Arguments: 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#27 ASC NULLS FIRST], [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (168) -+- * ColumnarToRow (167) - +- CometProject (166) - +- CometFilter (165) - +- CometScan parquet spark_catalog.default.date_dim (164) +BroadcastExchange (158) ++- * ColumnarToRow (157) + +- CometProject (156) + +- CometFilter (155) + +- CometScan parquet spark_catalog.default.date_dim (154) (unknown) Scan parquet spark_catalog.default.date_dim @@ -923,27 +883,27 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(165) CometFilter +(155) CometFilter Input [2]: [d_date_sk#25, d_year#259] Condition : ((isnotnull(d_year#259) AND (d_year#259 = 2001)) AND isnotnull(d_date_sk#25)) -(166) CometProject +(156) CometProject Input [2]: [d_date_sk#25, d_year#259] Arguments: [d_date_sk#25], [d_date_sk#25] -(167) ColumnarToRow [codegen id : 1] +(157) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(168) BroadcastExchange +(158) BroadcastExchange Input [1]: [d_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 42 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 97 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 133 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index cc38540a22..f02809572c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt @@ -2,242 +2,232 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Union WholeStageCodegen (8) HashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + InputAdapter + Exchange [i_item_id,ca_country,ca_state,ca_county] #1 + WholeStageCodegen (7) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #4 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (16) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state] #8 - RowToColumnar - WholeStageCodegen (15) - HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + Exchange [i_item_id,ca_country,ca_state] #8 + WholeStageCodegen (15) + HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 InputAdapter - ReusedExchange [cd_demo_sk] #5 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [cd_demo_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #9 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 WholeStageCodegen (24) HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country] #10 - RowToColumnar - WholeStageCodegen (23) - HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + Exchange [i_item_id,ca_country] #10 + WholeStageCodegen (23) + HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 InputAdapter - ReusedExchange [cd_demo_sk] #5 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [cd_demo_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #11 + WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk,ca_country] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 WholeStageCodegen (32) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #12 - RowToColumnar - WholeStageCodegen (31) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + Exchange [i_item_id] #12 + WholeStageCodegen (31) + HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 InputAdapter - ReusedExchange [cd_demo_sk] #5 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (28) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ReusedExchange [cd_demo_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #13 + WholeStageCodegen (28) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 WholeStageCodegen (40) HashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #14 - RowToColumnar - WholeStageCodegen (39) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + Exchange #14 + WholeStageCodegen (39) + HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 InputAdapter - ReusedExchange [cd_demo_sk] #5 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 InputAdapter - ReusedExchange [ca_address_sk] #13 + ReusedExchange [cd_demo_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [ca_address_sk] #13 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (38) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (38) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index 9405af5bd9..ad52796edc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -1,28 +1,24 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * ColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- RowToColumnar (18) - +- * HashAggregate (17) - +- * ColumnarToRow (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (20) ++- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -68,7 +64,7 @@ Join condition: None Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 29] +(10) ReusedExchange [Reuses operator id: 25] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -88,57 +84,45 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) RowToColumnar +(14) Exchange Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] - -(17) HashAggregate [codegen id : 4] +(15) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16] -(18) RowToColumnar -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] - -(19) CometColumnarExchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort +(16) Exchange Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16], [i_class#9 ASC NULLS FIRST] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) ColumnarToRow [codegen id : 5] +(17) Sort [codegen id : 5] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(22) Window +(18) Window Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(23) Project [codegen id : 6] +(19) Project [codegen id : 6] Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] -(24) TakeOrderedAndProject +(20) TakeOrderedAndProject Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (25) ++- * ColumnarToRow (24) + +- CometProject (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.date_dim (21) (unknown) Scan parquet spark_catalog.default.date_dim @@ -148,18 +132,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(26) CometFilter +(22) CometFilter Input [2]: [d_date_sk#11, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(27) CometProject +(23) CometProject Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(24) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(29) BroadcastExchange +(25) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index 0cc4015f20..2a2a392cd0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -4,41 +4,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (5) - ColumnarToRow + Sort [i_class] InputAdapter - CometSort [i_class] - CometColumnarExchange [i_class] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 + Exchange [i_class] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt index 0bca5ae8cd..bdfd6eee0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt @@ -1,28 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- * ColumnarToRow (22) - +- CometColumnarExchange (21) - +- RowToColumnar (20) - +- * HashAggregate (19) - +- * Expand (18) - +- * Project (17) - +- * BroadcastNestedLoopJoin Inner BuildRight (16) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometScan parquet spark_catalog.default.warehouse (13) +TakeOrderedAndProject (22) ++- * HashAggregate (21) + +- Exchange (20) + +- * HashAggregate (19) + +- * Expand (18) + +- * Project (17) + +- * BroadcastNestedLoopJoin Inner BuildRight (16) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometScan parquet spark_catalog.default.warehouse (13) (unknown) Scan parquet spark_catalog.default.inventory @@ -40,7 +38,7 @@ Condition : isnotnull(inv_item_sk#1) (3) ColumnarToRow [codegen id : 4] Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -(4) ReusedExchange [Reuses operator id: 29] +(4) ReusedExchange [Reuses operator id: 27] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 4] @@ -113,35 +111,29 @@ Functions [1]: [partial_avg(inv_quantity_on_hand#2)] Aggregate Attributes [2]: [sum#16, count#17] Results [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] -(20) RowToColumnar +(20) Exchange Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] +Arguments: hashpartitioning(i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometColumnarExchange -Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) ColumnarToRow [codegen id : 5] -Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] - -(23) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 5] Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] Keys [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] Functions [1]: [avg(inv_quantity_on_hand#2)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#20] Results [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, avg(inv_quantity_on_hand#2)#20 AS qoh#21] -(24) TakeOrderedAndProject +(22) TakeOrderedAndProject Input [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] Arguments: 100, [qoh#21 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#12 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_category#14 ASC NULLS FIRST], [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) (unknown) Scan parquet spark_catalog.default.date_dim @@ -151,18 +143,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(26) CometFilter +(24) CometFilter Input [2]: [d_date_sk#5, d_month_seq#22] Condition : (((isnotnull(d_month_seq#22) AND (d_month_seq#22 >= 1200)) AND (d_month_seq#22 <= 1211)) AND isnotnull(d_date_sk#5)) -(27) CometProject +(25) CometProject Input [2]: [d_date_sk#5, d_month_seq#22] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(29) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 2640e2dc58..63a428d4e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -1,43 +1,41 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 + InputAdapter + Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + BroadcastNestedLoopJoin + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.warehouse + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index 98d08fb2f2..e0a290cea7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -1,63 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (59) -+- Union (58) - :- * HashAggregate (25) - : +- * HashAggregate (24) - : +- * HashAggregate (23) - : +- * ColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- RowToColumnar (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.warehouse (13) +TakeOrderedAndProject (45) ++- Union (44) + :- * HashAggregate (23) + : +- * HashAggregate (22) + : +- * HashAggregate (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.item (7) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.warehouse (13) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * HashAggregate (25) + : +- ReusedExchange (24) :- * HashAggregate (33) - : +- * ColumnarToRow (32) - : +- CometColumnarExchange (31) - : +- RowToColumnar (30) - : +- * HashAggregate (29) - : +- * HashAggregate (28) - : +- * ColumnarToRow (27) - : +- ReusedExchange (26) - :- * HashAggregate (41) - : +- * ColumnarToRow (40) - : +- CometColumnarExchange (39) - : +- RowToColumnar (38) - : +- * HashAggregate (37) - : +- * HashAggregate (36) - : +- * ColumnarToRow (35) - : +- ReusedExchange (34) - :- * HashAggregate (49) - : +- * ColumnarToRow (48) - : +- CometColumnarExchange (47) - : +- RowToColumnar (46) - : +- * HashAggregate (45) - : +- * HashAggregate (44) - : +- * ColumnarToRow (43) - : +- ReusedExchange (42) - +- * HashAggregate (57) - +- * ColumnarToRow (56) - +- CometColumnarExchange (55) - +- RowToColumnar (54) - +- * HashAggregate (53) - +- * HashAggregate (52) - +- * ColumnarToRow (51) - +- ReusedExchange (50) + : +- Exchange (32) + : +- * HashAggregate (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + :- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * HashAggregate (35) + : +- ReusedExchange (34) + +- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * HashAggregate (40) + +- ReusedExchange (39) (unknown) Scan parquet spark_catalog.default.inventory @@ -75,7 +61,7 @@ Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (3) ColumnarToRow [codegen id : 4] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(4) ReusedExchange [Reuses operator id: 64] +(4) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 4] @@ -151,199 +137,157 @@ Functions [1]: [partial_avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [sum#13, count#14] Results [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(20) RowToColumnar +(20) Exchange Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] +Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, i_category#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometColumnarExchange -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) ColumnarToRow [codegen id : 5] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] - -(23) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 5] Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(24) HashAggregate [codegen id : 5] +(22) HashAggregate [codegen id : 5] Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#18] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [partial_avg(qoh#18)] Aggregate Attributes [2]: [sum#19, count#20] Results [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] -(25) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 5] Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(qoh#18)] Aggregate Attributes [1]: [avg(qoh#18)#23] Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(qoh#18)#23 AS qoh#24] -(26) ReusedExchange [Reuses operator id: 21] +(24) ReusedExchange [Reuses operator id: 20] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(27) ColumnarToRow [codegen id : 10] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] - -(28) HashAggregate [codegen id : 10] +(25) HashAggregate [codegen id : 10] Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] Results [4]: [i_product_name#11, i_brand#8, i_class#9, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(29) HashAggregate [codegen id : 10] +(26) HashAggregate [codegen id : 10] Input [4]: [i_product_name#11, i_brand#8, i_class#9, qoh#18] Keys [3]: [i_product_name#11, i_brand#8, i_class#9] Functions [1]: [partial_avg(qoh#18)] Aggregate Attributes [2]: [sum#25, count#26] Results [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] -(30) RowToColumnar +(27) Exchange Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] +Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(31) CometColumnarExchange -Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] -Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(32) ColumnarToRow [codegen id : 11] -Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] - -(33) HashAggregate [codegen id : 11] +(28) HashAggregate [codegen id : 11] Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] Keys [3]: [i_product_name#11, i_brand#8, i_class#9] Functions [1]: [avg(qoh#18)] Aggregate Attributes [1]: [avg(qoh#18)#29] Results [5]: [i_product_name#11, i_brand#8, i_class#9, null AS i_category#30, avg(qoh#18)#29 AS qoh#31] -(34) ReusedExchange [Reuses operator id: 21] +(29) ReusedExchange [Reuses operator id: 20] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(35) ColumnarToRow [codegen id : 16] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] - -(36) HashAggregate [codegen id : 16] +(30) HashAggregate [codegen id : 16] Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] Results [3]: [i_product_name#11, i_brand#8, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(37) HashAggregate [codegen id : 16] +(31) HashAggregate [codegen id : 16] Input [3]: [i_product_name#11, i_brand#8, qoh#18] Keys [2]: [i_product_name#11, i_brand#8] Functions [1]: [partial_avg(qoh#18)] Aggregate Attributes [2]: [sum#32, count#33] Results [4]: [i_product_name#11, i_brand#8, sum#34, count#35] -(38) RowToColumnar -Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] - -(39) CometColumnarExchange -Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] -Arguments: hashpartitioning(i_product_name#11, i_brand#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) ColumnarToRow [codegen id : 17] +(32) Exchange Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] +Arguments: hashpartitioning(i_product_name#11, i_brand#8, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(41) HashAggregate [codegen id : 17] +(33) HashAggregate [codegen id : 17] Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] Keys [2]: [i_product_name#11, i_brand#8] Functions [1]: [avg(qoh#18)] Aggregate Attributes [1]: [avg(qoh#18)#36] Results [5]: [i_product_name#11, i_brand#8, null AS i_class#37, null AS i_category#38, avg(qoh#18)#36 AS qoh#39] -(42) ReusedExchange [Reuses operator id: 21] +(34) ReusedExchange [Reuses operator id: 20] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(43) ColumnarToRow [codegen id : 22] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] - -(44) HashAggregate [codegen id : 22] +(35) HashAggregate [codegen id : 22] Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] Results [2]: [i_product_name#11, avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(45) HashAggregate [codegen id : 22] +(36) HashAggregate [codegen id : 22] Input [2]: [i_product_name#11, qoh#18] Keys [1]: [i_product_name#11] Functions [1]: [partial_avg(qoh#18)] Aggregate Attributes [2]: [sum#40, count#41] Results [3]: [i_product_name#11, sum#42, count#43] -(46) RowToColumnar -Input [3]: [i_product_name#11, sum#42, count#43] - -(47) CometColumnarExchange +(37) Exchange Input [3]: [i_product_name#11, sum#42, count#43] -Arguments: hashpartitioning(i_product_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(i_product_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(48) ColumnarToRow [codegen id : 23] -Input [3]: [i_product_name#11, sum#42, count#43] - -(49) HashAggregate [codegen id : 23] +(38) HashAggregate [codegen id : 23] Input [3]: [i_product_name#11, sum#42, count#43] Keys [1]: [i_product_name#11] Functions [1]: [avg(qoh#18)] Aggregate Attributes [1]: [avg(qoh#18)#44] Results [5]: [i_product_name#11, null AS i_brand#45, null AS i_class#46, null AS i_category#47, avg(qoh#18)#44 AS qoh#48] -(50) ReusedExchange [Reuses operator id: 21] +(39) ReusedExchange [Reuses operator id: 20] Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(51) ColumnarToRow [codegen id : 28] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] - -(52) HashAggregate [codegen id : 28] +(40) HashAggregate [codegen id : 28] Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(inv_quantity_on_hand#3)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] Results [1]: [avg(inv_quantity_on_hand#3)#17 AS qoh#18] -(53) HashAggregate [codegen id : 28] +(41) HashAggregate [codegen id : 28] Input [1]: [qoh#18] Keys: [] Functions [1]: [partial_avg(qoh#18)] Aggregate Attributes [2]: [sum#49, count#50] Results [2]: [sum#51, count#52] -(54) RowToColumnar -Input [2]: [sum#51, count#52] - -(55) CometColumnarExchange -Input [2]: [sum#51, count#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(56) ColumnarToRow [codegen id : 29] +(42) Exchange Input [2]: [sum#51, count#52] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(57) HashAggregate [codegen id : 29] +(43) HashAggregate [codegen id : 29] Input [2]: [sum#51, count#52] Keys: [] Functions [1]: [avg(qoh#18)] Aggregate Attributes [1]: [avg(qoh#18)#53] Results [5]: [null AS i_product_name#54, null AS i_brand#55, null AS i_class#56, null AS i_category#57, avg(qoh#18)#53 AS qoh#58] -(58) Union +(44) Union -(59) TakeOrderedAndProject +(45) TakeOrderedAndProject Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (64) -+- * ColumnarToRow (63) - +- CometProject (62) - +- CometFilter (61) - +- CometScan parquet spark_catalog.default.date_dim (60) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim @@ -353,18 +297,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(61) CometFilter +(47) CometFilter Input [2]: [d_date_sk#6, d_month_seq#59] Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1212)) AND (d_month_seq#59 <= 1223)) AND isnotnull(d_date_sk#6)) -(62) CometProject +(48) CometProject Input [2]: [d_date_sk#6, d_month_seq#59] Arguments: [d_date_sk#6], [d_date_sk#6] -(63) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(64) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index a614a9093d..a8d71b06ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -4,91 +4,77 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 + InputAdapter + Exchange [i_product_name,i_brand,i_class,i_category] #1 + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (11) HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class] #5 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - ColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + InputAdapter + Exchange [i_product_name,i_brand,i_class] #5 + WholeStageCodegen (10) + HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 WholeStageCodegen (17) HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand] #6 - RowToColumnar - WholeStageCodegen (16) - HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - ColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + InputAdapter + Exchange [i_product_name,i_brand] #6 + WholeStageCodegen (16) + HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 WholeStageCodegen (23) HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name] #7 - RowToColumnar - WholeStageCodegen (22) - HashAggregate [i_product_name,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - ColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + InputAdapter + Exchange [i_product_name] #7 + WholeStageCodegen (22) + HashAggregate [i_product_name,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 WholeStageCodegen (29) HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #8 - RowToColumnar - WholeStageCodegen (28) - HashAggregate [qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - ColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + InputAdapter + Exchange #8 + WholeStageCodegen (28) + HashAggregate [qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index 325b853594..bc8b8657e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -1,57 +1,52 @@ == Physical Plan == -* ColumnarToRow (53) -+- CometSort (52) - +- CometColumnarExchange (51) - +- RowToColumnar (50) - +- * Filter (49) - +- * HashAggregate (48) - +- * ColumnarToRow (47) - +- CometColumnarExchange (46) - +- RowToColumnar (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- * ColumnarToRow (42) - +- CometColumnarExchange (41) - +- RowToColumnar (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometColumnarExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometColumnarExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.item (21) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_address (33) +* Sort (48) ++- Exchange (47) + +- * Filter (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store_returns (7) + : : : +- BroadcastExchange (19) + : : : +- * ColumnarToRow (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.store (15) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometFilter (23) + : : +- CometScan parquet spark_catalog.default.item (22) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer (28) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer_address (34) (unknown) Scan parquet spark_catalog.default.store_sales @@ -69,13 +64,16 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) CometColumnarExchange +(4) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(5) CometSort +(5) Exchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(6) Sort [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] @@ -84,33 +82,34 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(7) CometFilter +(8) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(8) CometProject +(9) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(9) CometColumnarExchange +(10) ColumnarToRow [codegen id : 3] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) CometSort +(11) Exchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(12) Sort [codegen id : 4] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(13) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(13) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(14) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] (unknown) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] @@ -119,28 +118,28 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(15) CometFilter +(16) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(16) CometProject +(17) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(17) ColumnarToRow [codegen id : 1] +(18) ColumnarToRow [codegen id : 5] Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) BroadcastExchange +(19) BroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(21) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -151,24 +150,24 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(22) CometFilter +(23) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) -(23) ColumnarToRow [codegen id : 2] +(24) ColumnarToRow [codegen id : 6] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(24) BroadcastExchange +(25) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(27) Project [codegen id : 9] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] @@ -179,24 +178,24 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(28) CometFilter +(29) CometFilter Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Condition : ((isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) AND isnotnull(c_birth_country#25)) -(29) ColumnarToRow [codegen id : 3] +(30) ColumnarToRow [codegen id : 7] Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(30) BroadcastExchange +(31) BroadcastExchange Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(31) BroadcastHashJoin [codegen id : 5] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(32) Project [codegen id : 5] +(33) Project [codegen id : 9] Output [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] @@ -207,166 +206,141 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(34) CometFilter +(35) CometFilter Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_country#29)) AND isnotnull(ca_zip#28)) -(35) ColumnarToRow [codegen id : 4] +(36) ColumnarToRow [codegen id : 8] Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(36) BroadcastExchange +(37) BroadcastExchange Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 5] +(38) BroadcastHashJoin [codegen id : 9] Left keys [3]: [c_current_addr_sk#22, c_birth_country#25, s_zip#14] Right keys [3]: [ca_address_sk#26, upper(ca_country#29), ca_zip#28] Join type: Inner Join condition: None -(38) Project [codegen id : 5] +(39) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25, ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(39) HashAggregate [codegen id : 5] +(40) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#30] Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] -(40) RowToColumnar +(41) Exchange Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(41) CometColumnarExchange -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(42) ColumnarToRow [codegen id : 6] -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] - -(43) HashAggregate [codegen id : 6] +(42) HashAggregate [codegen id : 10] Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] -(44) HashAggregate [codegen id : 6] +(43) HashAggregate [codegen id : 10] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, netpaid#33] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] Functions [1]: [partial_sum(netpaid#33)] Aggregate Attributes [2]: [sum#34, isEmpty#35] Results [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] -(45) RowToColumnar -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] - -(46) CometColumnarExchange -Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(47) ColumnarToRow [codegen id : 7] +(44) Exchange Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(48) HashAggregate [codegen id : 7] +(45) HashAggregate [codegen id : 11] Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] Functions [1]: [sum(netpaid#33)] Aggregate Attributes [1]: [sum(netpaid#33)#38] Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, sum(netpaid#33)#38 AS paid#39] -(49) Filter [codegen id : 7] +(46) Filter [codegen id : 11] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] Condition : (isnotnull(paid#39) AND (cast(paid#39 as decimal(33,8)) > cast(Subquery scalar-subquery#40, [id=#41] as decimal(33,8)))) -(50) RowToColumnar +(47) Exchange Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] +Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(51) CometColumnarExchange -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(52) CometSort -Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] -Arguments: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39], [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] - -(53) ColumnarToRow [codegen id : 8] +(48) Sort [codegen id : 12] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] +Arguments: [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* HashAggregate (85) -+- * ColumnarToRow (84) - +- CometColumnarExchange (83) - +- RowToColumnar (82) - +- * HashAggregate (81) - +- * HashAggregate (80) - +- * ColumnarToRow (79) - +- CometColumnarExchange (78) - +- RowToColumnar (77) - +- * HashAggregate (76) - +- * Project (75) - +- * BroadcastHashJoin Inner BuildRight (74) - :- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (69) - : : +- * BroadcastHashJoin Inner BuildRight (68) - : : :- * Project (63) - : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : :- * ColumnarToRow (60) - : : : : +- CometProject (59) - : : : : +- CometSortMergeJoin (58) - : : : : :- CometSort (55) - : : : : : +- ReusedExchange (54) - : : : : +- CometSort (57) - : : : : +- ReusedExchange (56) - : : : +- ReusedExchange (61) - : : +- BroadcastExchange (67) - : : +- * ColumnarToRow (66) - : : +- CometFilter (65) - : : +- CometScan parquet spark_catalog.default.item (64) - : +- ReusedExchange (70) - +- ReusedExchange (73) - - -(54) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +* HashAggregate (75) ++- Exchange (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- Exchange (71) + +- * HashAggregate (70) + +- * Project (69) + +- * BroadcastHashJoin Inner BuildRight (68) + :- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * Project (57) + : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : :- * Project (54) + : : : : +- * SortMergeJoin Inner (53) + : : : : :- * Sort (50) + : : : : : +- ReusedExchange (49) + : : : : +- * Sort (52) + : : : : +- ReusedExchange (51) + : : : +- ReusedExchange (55) + : : +- BroadcastExchange (61) + : : +- * ColumnarToRow (60) + : : +- CometFilter (59) + : : +- CometScan parquet spark_catalog.default.item (58) + : +- ReusedExchange (64) + +- ReusedExchange (67) + + +(49) ReusedExchange [Reuses operator id: 5] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(55) CometSort +(50) Sort [codegen id : 2] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 -(56) ReusedExchange [Reuses operator id: 9] +(51) ReusedExchange [Reuses operator id: 11] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(57) CometSort +(52) Sort [codegen id : 4] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(58) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(53) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(59) CometProject +(54) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(60) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(61) ReusedExchange [Reuses operator id: 18] +(55) ReusedExchange [Reuses operator id: 19] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(62) BroadcastHashJoin [codegen id : 5] +(56) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(63) Project [codegen id : 5] +(57) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -377,95 +351,83 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(65) CometFilter +(59) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : isnotnull(i_item_sk#15) -(66) ColumnarToRow [codegen id : 2] +(60) ColumnarToRow [codegen id : 6] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(67) BroadcastExchange +(61) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(68) BroadcastHashJoin [codegen id : 5] +(62) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(69) Project [codegen id : 5] +(63) Project [codegen id : 9] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(70) ReusedExchange [Reuses operator id: 30] +(64) ReusedExchange [Reuses operator id: 31] Output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(71) BroadcastHashJoin [codegen id : 5] +(65) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(72) Project [codegen id : 5] +(66) Project [codegen id : 9] Output [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(73) ReusedExchange [Reuses operator id: 36] +(67) ReusedExchange [Reuses operator id: 37] Output [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(74) BroadcastHashJoin [codegen id : 5] +(68) BroadcastHashJoin [codegen id : 9] Left keys [3]: [c_current_addr_sk#22, c_birth_country#25, s_zip#14] Right keys [3]: [ca_address_sk#26, upper(ca_country#29), ca_zip#28] Join type: Inner Join condition: None -(75) Project [codegen id : 5] +(69) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25, ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(76) HashAggregate [codegen id : 5] +(70) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#42] Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] -(77) RowToColumnar +(71) Exchange Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(78) CometColumnarExchange -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(79) ColumnarToRow [codegen id : 6] -Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] - -(80) HashAggregate [codegen id : 6] +(72) HashAggregate [codegen id : 10] Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#43] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] -(81) HashAggregate [codegen id : 6] +(73) HashAggregate [codegen id : 10] Input [1]: [netpaid#33] Keys: [] Functions [1]: [partial_avg(netpaid#33)] Aggregate Attributes [2]: [sum#44, count#45] Results [2]: [sum#46, count#47] -(82) RowToColumnar -Input [2]: [sum#46, count#47] - -(83) CometColumnarExchange -Input [2]: [sum#46, count#47] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(84) ColumnarToRow [codegen id : 7] +(74) Exchange Input [2]: [sum#46, count#47] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(85) HashAggregate [codegen id : 7] +(75) HashAggregate [codegen id : 11] Input [2]: [sum#46, count#47] Keys: [] Functions [1]: [avg(netpaid#33)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index 08ef232256..7024f439fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -1,118 +1,122 @@ -WholeStageCodegen (8) - ColumnarToRow +WholeStageCodegen (12) + Sort [c_last_name,c_first_name,s_store_name] InputAdapter - CometSort [c_last_name,c_first_name,s_store_name] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - RowToColumnar - WholeStageCodegen (7) - Filter [paid] - Subquery #1 - WholeStageCodegen (7) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #10 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Exchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (11) + Filter [paid] + Subquery #1 + WholeStageCodegen (11) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + Exchange #10 + WholeStageCodegen (10) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_color,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number] #5 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) + BroadcastExchange #12 + WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + InputAdapter + ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #2 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_country,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt index 5567524669..54aadf2cf5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt @@ -1,83 +1,77 @@ == Physical Plan == -TakeOrderedAndProject (79) -+- Union (78) - :- * HashAggregate (30) - : +- * ColumnarToRow (29) - : +- CometColumnarExchange (28) - : +- RowToColumnar (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.item (20) - :- * HashAggregate (54) - : +- * ColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- RowToColumnar (51) - : +- * HashAggregate (50) - : +- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (46) - : : +- * BroadcastHashJoin Inner BuildRight (45) - : : :- * Project (39) - : : : +- * BroadcastHashJoin Inner BuildRight (38) - : : : :- * Project (36) - : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : :- * ColumnarToRow (33) - : : : : : +- CometFilter (32) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (31) - : : : : +- ReusedExchange (34) - : : : +- ReusedExchange (37) - : : +- BroadcastExchange (44) - : : +- * ColumnarToRow (43) - : : +- CometProject (42) - : : +- CometFilter (41) - : : +- CometScan parquet spark_catalog.default.store (40) - : +- ReusedExchange (47) - +- * HashAggregate (77) - +- * ColumnarToRow (76) - +- CometColumnarExchange (75) - +- RowToColumnar (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * ColumnarToRow (57) - : : : : +- CometFilter (56) - : : : : +- CometScan parquet spark_catalog.default.store_sales (55) - : : : +- ReusedExchange (58) - : : +- ReusedExchange (61) - : +- ReusedExchange (64) - +- BroadcastExchange (70) - +- * ColumnarToRow (69) - +- CometFilter (68) - +- CometScan parquet spark_catalog.default.item (67) +TakeOrderedAndProject (73) ++- Union (72) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (17) + : : +- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.store (14) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + :- * HashAggregate (50) + : +- Exchange (49) + : +- * HashAggregate (48) + : +- * Project (47) + : +- * BroadcastHashJoin Inner BuildRight (46) + : :- * Project (44) + : : +- * BroadcastHashJoin Inner BuildRight (43) + : : :- * Project (37) + : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : :- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * ColumnarToRow (31) + : : : : : +- CometFilter (30) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) + : : : : +- ReusedExchange (32) + : : : +- ReusedExchange (35) + : : +- BroadcastExchange (42) + : : +- * ColumnarToRow (41) + : : +- CometProject (40) + : : +- CometFilter (39) + : : +- CometScan parquet spark_catalog.default.store (38) + : +- ReusedExchange (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * Project (56) + : : : +- * BroadcastHashJoin Inner BuildRight (55) + : : : :- * ColumnarToRow (53) + : : : : +- CometFilter (52) + : : : : +- CometScan parquet spark_catalog.default.store_sales (51) + : : : +- ReusedExchange (54) + : : +- ReusedExchange (57) + : +- ReusedExchange (60) + +- BroadcastExchange (66) + +- * ColumnarToRow (65) + +- CometFilter (64) + +- CometScan parquet spark_catalog.default.item (63) (unknown) Scan parquet spark_catalog.default.store_sales @@ -127,7 +121,7 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(11) ReusedExchange [Reuses operator id: 84] +(11) ReusedExchange [Reuses operator id: 78] Output [1]: [d_date_sk#14] (12) BroadcastHashJoin [codegen id : 5] @@ -203,17 +197,11 @@ Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), parti Aggregate Attributes [8]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30] Results [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] -(27) RowToColumnar +(27) Exchange Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] +Arguments: hashpartitioning(i_item_id#18, s_state#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) CometColumnarExchange -Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] -Arguments: hashpartitioning(i_item_id#18, s_state#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) ColumnarToRow [codegen id : 6] -Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] - -(30) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 6] Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] Keys [2]: [i_item_id#18, s_state#16] Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] @@ -228,36 +216,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(32) CometFilter +(30) CometFilter Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) -(33) ColumnarToRow [codegen id : 11] +(31) ColumnarToRow [codegen id : 11] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(34) ReusedExchange [Reuses operator id: 8] +(32) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#10] -(35) BroadcastHashJoin [codegen id : 11] +(33) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(36) Project [codegen id : 11] +(34) Project [codegen id : 11] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(37) ReusedExchange [Reuses operator id: 84] +(35) ReusedExchange [Reuses operator id: 78] Output [1]: [d_date_sk#14] -(38) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(39) Project [codegen id : 11] +(37) Project [codegen id : 11] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -268,62 +256,56 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(41) CometFilter +(39) CometFilter Input [2]: [s_store_sk#15, s_state#16] Condition : ((isnotnull(s_state#16) AND (s_state#16 = TN)) AND isnotnull(s_store_sk#15)) -(42) CometProject +(40) CometProject Input [2]: [s_store_sk#15, s_state#16] Arguments: [s_store_sk#15], [s_store_sk#15] -(43) ColumnarToRow [codegen id : 9] +(41) ColumnarToRow [codegen id : 9] Input [1]: [s_store_sk#15] -(44) BroadcastExchange +(42) BroadcastExchange Input [1]: [s_store_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(45) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(46) Project [codegen id : 11] +(44) Project [codegen id : 11] Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15] -(47) ReusedExchange [Reuses operator id: 23] +(45) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#17, i_item_id#18] -(48) BroadcastHashJoin [codegen id : 11] +(46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(49) Project [codegen id : 11] +(47) Project [codegen id : 11] Output [5]: [i_item_id#18, ss_quantity#4 AS agg1#19, ss_list_price#5 AS agg2#20, ss_coupon_amt#7 AS agg3#21, ss_sales_price#6 AS agg4#22] Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#17, i_item_id#18] -(50) HashAggregate [codegen id : 11] +(48) HashAggregate [codegen id : 11] Input [5]: [i_item_id#18, agg1#19, agg2#20, agg3#21, agg4#22] Keys [1]: [i_item_id#18] Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), partial_avg(UnscaledValue(agg3#21)), partial_avg(UnscaledValue(agg4#22))] Aggregate Attributes [8]: [sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56] Results [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] -(51) RowToColumnar -Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] - -(52) CometColumnarExchange +(49) Exchange Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(53) ColumnarToRow [codegen id : 12] -Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] - -(54) HashAggregate [codegen id : 12] +(50) HashAggregate [codegen id : 12] Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] Keys [1]: [i_item_id#18] Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] @@ -338,49 +320,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(56) CometFilter +(52) CometFilter Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) -(57) ColumnarToRow [codegen id : 17] +(53) ColumnarToRow [codegen id : 17] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(58) ReusedExchange [Reuses operator id: 8] +(54) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#10] -(59) BroadcastHashJoin [codegen id : 17] +(55) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(60) Project [codegen id : 17] +(56) Project [codegen id : 17] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(61) ReusedExchange [Reuses operator id: 84] +(57) ReusedExchange [Reuses operator id: 78] Output [1]: [d_date_sk#14] -(62) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(59) Project [codegen id : 17] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -(64) ReusedExchange [Reuses operator id: 44] +(60) ReusedExchange [Reuses operator id: 42] Output [1]: [s_store_sk#15] -(65) BroadcastHashJoin [codegen id : 17] +(61) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(66) Project [codegen id : 17] +(62) Project [codegen id : 17] Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15] @@ -391,65 +373,59 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(68) CometFilter +(64) CometFilter Input [1]: [i_item_sk#17] Condition : isnotnull(i_item_sk#17) -(69) ColumnarToRow [codegen id : 16] +(65) ColumnarToRow [codegen id : 16] Input [1]: [i_item_sk#17] -(70) BroadcastExchange +(66) BroadcastExchange Input [1]: [i_item_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(71) BroadcastHashJoin [codegen id : 17] +(67) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(72) Project [codegen id : 17] +(68) Project [codegen id : 17] Output [4]: [ss_quantity#4 AS agg1#19, ss_list_price#5 AS agg2#20, ss_coupon_amt#7 AS agg3#21, ss_sales_price#6 AS agg4#22] Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#17] -(73) HashAggregate [codegen id : 17] +(69) HashAggregate [codegen id : 17] Input [4]: [agg1#19, agg2#20, agg3#21, agg4#22] Keys: [] Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), partial_avg(UnscaledValue(agg3#21)), partial_avg(UnscaledValue(agg4#22))] Aggregate Attributes [8]: [sum#76, count#77, sum#78, count#79, sum#80, count#81, sum#82, count#83] Results [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -(74) RowToColumnar -Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] - -(75) CometColumnarExchange -Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(76) ColumnarToRow [codegen id : 18] +(70) Exchange Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(77) HashAggregate [codegen id : 18] +(71) HashAggregate [codegen id : 18] Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] Keys: [] Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] Aggregate Attributes [4]: [avg(agg1#19)#92, avg(UnscaledValue(agg2#20))#93, avg(UnscaledValue(agg3#21))#94, avg(UnscaledValue(agg4#22))#95] Results [7]: [null AS i_item_id#96, null AS s_state#97, 1 AS g_state#98, avg(agg1#19)#92 AS agg1#99, cast((avg(UnscaledValue(agg2#20))#93 / 100.0) as decimal(11,6)) AS agg2#100, cast((avg(UnscaledValue(agg3#21))#94 / 100.0) as decimal(11,6)) AS agg3#101, cast((avg(UnscaledValue(agg4#22))#95 / 100.0) as decimal(11,6)) AS agg4#102] -(78) Union +(72) Union -(79) TakeOrderedAndProject +(73) TakeOrderedAndProject Input [7]: [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] Arguments: 100, [i_item_id#18 ASC NULLS FIRST, s_state#16 ASC NULLS FIRST], [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (84) -+- * ColumnarToRow (83) - +- CometProject (82) - +- CometFilter (81) - +- CometScan parquet spark_catalog.default.date_dim (80) +BroadcastExchange (78) ++- * ColumnarToRow (77) + +- CometProject (76) + +- CometFilter (75) + +- CometScan parquet spark_catalog.default.date_dim (74) (unknown) Scan parquet spark_catalog.default.date_dim @@ -459,23 +435,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter +(75) CometFilter Input [2]: [d_date_sk#14, d_year#103] Condition : ((isnotnull(d_year#103) AND (d_year#103 = 1998)) AND isnotnull(d_date_sk#14)) -(82) CometProject +(76) CometProject Input [2]: [d_date_sk#14, d_year#103] Arguments: [d_date_sk#14], [d_date_sk#14] -(83) ColumnarToRow [codegen id : 1] +(77) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(84) BroadcastExchange +(78) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt index 3a64f1517d..32f003798d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt @@ -2,122 +2,116 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Union WholeStageCodegen (6) HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,s_state] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + InputAdapter + Exchange [i_item_id,s_state] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (12) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #6 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk] #3 + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (11) + HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [cd_demo_sk] #3 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + BroadcastExchange #7 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - ColumnarToRow - InputAdapter - CometColumnarExchange #8 - RowToColumnar - WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk] #3 + InputAdapter + Exchange #8 + WholeStageCodegen (17) + HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [cd_demo_sk] #3 InputAdapter - ReusedExchange [s_store_sk] #7 + ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] + ReusedExchange [s_store_sk] #7 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt index b80a05efb2..a86edcfa3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt @@ -1,40 +1,36 @@ == Physical Plan == -* ColumnarToRow (36) -+- CometSort (35) - +- CometColumnarExchange (34) - +- RowToColumnar (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (26) - : +- * HashAggregate (25) - : +- * ColumnarToRow (24) - : +- CometColumnarExchange (23) - : +- RowToColumnar (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * ColumnarToRow (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.customer (27) +* Sort (32) ++- Exchange (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * Filter (24) + : +- * HashAggregate (23) + : +- Exchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (28) + +- * ColumnarToRow (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (unknown) Scan parquet spark_catalog.default.store_sales @@ -52,7 +48,7 @@ Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnu (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 41] +(4) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -136,24 +132,18 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#14] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) RowToColumnar +(22) Exchange Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(23) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) ColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(25) HashAggregate [codegen id : 6] +(23) HashAggregate [codegen id : 6] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(26) Filter [codegen id : 6] +(24) Filter [codegen id : 6] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) @@ -164,49 +154,43 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(28) CometFilter +(26) CometFilter Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Condition : isnotnull(c_customer_sk#18) -(29) ColumnarToRow [codegen id : 5] +(27) ColumnarToRow [codegen id : 5] Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(30) BroadcastExchange +(28) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(30) Project [codegen id : 6] Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -(33) RowToColumnar -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] - -(34) CometColumnarExchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(35) CometSort +(31) Exchange Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17], [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] +Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(36) ColumnarToRow [codegen id : 7] +(32) Sort [codegen id : 7] Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (41) -+- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (unknown) Scan parquet spark_catalog.default.date_dim @@ -216,18 +200,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(38) CometFilter +(34) CometFilter Input [3]: [d_date_sk#7, d_year#23, d_dom#24] Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(39) CometProject +(35) CometProject Input [3]: [d_date_sk#7, d_year#23, d_dom#24] Arguments: [d_date_sk#7], [d_date_sk#7] -(40) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(41) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index 9fb9e6a631..b473e48921 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -1,60 +1,56 @@ WholeStageCodegen (7) - ColumnarToRow + Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - RowToColumnar - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 + Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + WholeStageCodegen (6) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index a0653a4981..e723b6c0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -1,48 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * ColumnarToRow (42) - +- CometColumnarExchange (41) - +- RowToColumnar (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * Filter (25) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_demographics (33) (unknown) Scan parquet spark_catalog.default.customer @@ -69,7 +67,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) ReusedExchange [Reuses operator id: 49] +(6) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#9] (7) BroadcastHashJoin [codegen id : 2] @@ -102,7 +100,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(13) ReusedExchange [Reuses operator id: 49] +(13) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#13] (14) BroadcastHashJoin [codegen id : 4] @@ -135,7 +133,7 @@ ReadSchema: struct (19) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -(20) ReusedExchange [Reuses operator id: 49] +(20) ReusedExchange [Reuses operator id: 47] Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 6] @@ -229,35 +227,29 @@ Functions [10]: [partial_count(1), partial_avg(cd_dep_count#23), partial_max(cd_ Aggregate Attributes [13]: [count#26, sum#27, count#28, max#29, sum#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38] Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -(40) RowToColumnar +(40) Exchange Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] +Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) CometColumnarExchange -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) ColumnarToRow [codegen id : 10] -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] - -(43) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 10] Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] -(44) TakeOrderedAndProject +(42) TakeOrderedAndProject Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) (unknown) Scan parquet spark_catalog.default.date_dim @@ -267,18 +259,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(44) CometFilter Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) -(47) CometProject +(45) CometProject Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] Arguments: [d_date_sk#9], [d_date_sk#9] -(48) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index 7e3ae715aa..dc724ca919 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -1,76 +1,74 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] WholeStageCodegen (10) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index d9e550f2fc..29e62c0b79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -1,46 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- * ColumnarToRow (40) - +- CometColumnarExchange (39) - +- RowToColumnar (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.customer_demographics (31) +TakeOrderedAndProject (40) ++- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : +- BroadcastExchange (9) + : : : +- * Project (8) + : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : :- * ColumnarToRow (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (6) + : : +- BroadcastExchange (22) + : : +- Union (21) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * ColumnarToRow (17) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (18) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.customer_address (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.customer_demographics (31) (unknown) Scan parquet spark_catalog.default.customer @@ -67,7 +65,7 @@ ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) ReusedExchange [Reuses operator id: 47] +(6) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#7] (7) BroadcastHashJoin [codegen id : 2] @@ -100,7 +98,7 @@ ReadSchema: struct (12) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -(13) ReusedExchange [Reuses operator id: 47] +(13) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#11] (14) BroadcastHashJoin [codegen id : 4] @@ -123,7 +121,7 @@ ReadSchema: struct (17) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(18) ReusedExchange [Reuses operator id: 47] +(18) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#16] (19) BroadcastHashJoin [codegen id : 6] @@ -215,35 +213,29 @@ Functions [10]: [partial_count(1), partial_avg(cd_dep_count#23), partial_max(cd_ Aggregate Attributes [13]: [count#26, sum#27, count#28, max#29, sum#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38] Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -(38) RowToColumnar +(38) Exchange Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] +Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(39) CometColumnarExchange -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) ColumnarToRow [codegen id : 10] -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] - -(41) HashAggregate [codegen id : 10] +(39) HashAggregate [codegen id : 10] Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] -(42) TakeOrderedAndProject +(40) TakeOrderedAndProject Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) (unknown) Scan parquet spark_catalog.default.date_dim @@ -253,18 +245,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(42) CometFilter Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 1999)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#7)) -(45) CometProject +(43) CometProject Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] Arguments: [d_date_sk#7], [d_date_sk#7] -(46) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(47) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index 524a6ed84e..e5cb940552 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -1,73 +1,71 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] WholeStageCodegen (10) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customsk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,customsk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + BroadcastExchange #4 + Union + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) + BroadcastExchange #5 + WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt index a87fa1adc0..2519f23a25 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt @@ -1,55 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- Window (49) - +- * ColumnarToRow (48) - +- CometSort (47) - +- CometColumnarExchange (46) - +- CometHashAggregate (45) - +- CometColumnarExchange (44) - +- RowToColumnar (43) - +- * HashAggregate (42) - +- Union (41) - :- * HashAggregate (24) - : +- * ColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- RowToColumnar (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.store (13) - :- * HashAggregate (32) - : +- * ColumnarToRow (31) - : +- CometColumnarExchange (30) - : +- RowToColumnar (29) - : +- * HashAggregate (28) - : +- * HashAggregate (27) - : +- * ColumnarToRow (26) - : +- ReusedExchange (25) - +- * HashAggregate (40) - +- * ColumnarToRow (39) - +- CometColumnarExchange (38) - +- RowToColumnar (37) - +- * HashAggregate (36) - +- * HashAggregate (35) - +- * ColumnarToRow (34) - +- ReusedExchange (33) +TakeOrderedAndProject (41) ++- * Project (40) + +- Window (39) + +- * Sort (38) + +- Exchange (37) + +- * HashAggregate (36) + +- Exchange (35) + +- * HashAggregate (34) + +- Union (33) + :- * HashAggregate (22) + : +- Exchange (21) + : +- * HashAggregate (20) + : +- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.item (7) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.store (13) + :- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * HashAggregate (24) + : +- ReusedExchange (23) + +- * HashAggregate (32) + +- Exchange (31) + +- * HashAggregate (30) + +- * HashAggregate (29) + +- ReusedExchange (28) (unknown) Scan parquet spark_catalog.default.store_sales @@ -67,7 +57,7 @@ Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 56] +(4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#7] (5) BroadcastHashJoin [codegen id : 4] @@ -147,149 +137,121 @@ Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(Unscale Aggregate Attributes [2]: [sum#13, sum#14] Results [4]: [i_category#10, i_class#9, sum#15, sum#16] -(21) RowToColumnar +(21) Exchange Input [4]: [i_category#10, i_class#9, sum#15, sum#16] +Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) CometColumnarExchange -Input [4]: [i_category#10, i_class#9, sum#15, sum#16] -Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) ColumnarToRow [codegen id : 5] -Input [4]: [i_category#10, i_class#9, sum#15, sum#16] - -(24) HashAggregate [codegen id : 5] +(22) HashAggregate [codegen id : 5] Input [4]: [i_category#10, i_class#9, sum#15, sum#16] Keys [2]: [i_category#10, i_class#9] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#17, sum(UnscaledValue(ss_ext_sales_price#3))#18] Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#17,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2)) as decimal(38,20)) AS gross_margin#19, i_category#10, i_class#9, 0 AS t_category#20, 0 AS t_class#21, 0 AS lochierarchy#22] -(25) ReusedExchange [Reuses operator id: 22] +(23) ReusedExchange [Reuses operator id: 21] Output [4]: [i_category#10, i_class#9, sum#23, sum#24] -(26) ColumnarToRow [codegen id : 10] -Input [4]: [i_category#10, i_class#9, sum#23, sum#24] - -(27) HashAggregate [codegen id : 10] +(24) HashAggregate [codegen id : 10] Input [4]: [i_category#10, i_class#9, sum#23, sum#24] Keys [2]: [i_category#10, i_class#9] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#25, sum(UnscaledValue(ss_ext_sales_price#3))#26] Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#25,17,2) AS ss_net_profit#27, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#26,17,2) AS ss_ext_sales_price#28, i_category#10] -(28) HashAggregate [codegen id : 10] +(25) HashAggregate [codegen id : 10] Input [3]: [ss_net_profit#27, ss_ext_sales_price#28, i_category#10] Keys [1]: [i_category#10] Functions [2]: [partial_sum(ss_net_profit#27), partial_sum(ss_ext_sales_price#28)] Aggregate Attributes [4]: [sum#29, isEmpty#30, sum#31, isEmpty#32] Results [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] -(29) RowToColumnar +(26) Exchange Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] +Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(30) CometColumnarExchange -Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(31) ColumnarToRow [codegen id : 11] -Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] - -(32) HashAggregate [codegen id : 11] +(27) HashAggregate [codegen id : 11] Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] Keys [1]: [i_category#10] Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)] Aggregate Attributes [2]: [sum(ss_net_profit#27)#37, sum(ss_ext_sales_price#28)#38] Results [6]: [cast((sum(ss_net_profit#27)#37 / sum(ss_ext_sales_price#28)#38) as decimal(38,20)) AS gross_margin#39, i_category#10, null AS i_class#40, 0 AS t_category#41, 1 AS t_class#42, 1 AS lochierarchy#43] -(33) ReusedExchange [Reuses operator id: 22] +(28) ReusedExchange [Reuses operator id: 21] Output [4]: [i_category#10, i_class#9, sum#44, sum#45] -(34) ColumnarToRow [codegen id : 16] -Input [4]: [i_category#10, i_class#9, sum#44, sum#45] - -(35) HashAggregate [codegen id : 16] +(29) HashAggregate [codegen id : 16] Input [4]: [i_category#10, i_class#9, sum#44, sum#45] Keys [2]: [i_category#10, i_class#9] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#25, sum(UnscaledValue(ss_ext_sales_price#3))#26] Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#25,17,2) AS ss_net_profit#27, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#26,17,2) AS ss_ext_sales_price#28] -(36) HashAggregate [codegen id : 16] +(30) HashAggregate [codegen id : 16] Input [2]: [ss_net_profit#27, ss_ext_sales_price#28] Keys: [] Functions [2]: [partial_sum(ss_net_profit#27), partial_sum(ss_ext_sales_price#28)] Aggregate Attributes [4]: [sum#46, isEmpty#47, sum#48, isEmpty#49] Results [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] -(37) RowToColumnar -Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] - -(38) CometColumnarExchange -Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(39) ColumnarToRow [codegen id : 17] +(31) Exchange Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(40) HashAggregate [codegen id : 17] +(32) HashAggregate [codegen id : 17] Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] Keys: [] Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)] Aggregate Attributes [2]: [sum(ss_net_profit#27)#54, sum(ss_ext_sales_price#28)#55] Results [6]: [cast((sum(ss_net_profit#27)#54 / sum(ss_ext_sales_price#28)#55) as decimal(38,20)) AS gross_margin#56, null AS i_category#57, null AS i_class#58, 1 AS t_category#59, 1 AS t_class#60, 2 AS lochierarchy#61] -(41) Union +(33) Union -(42) HashAggregate [codegen id : 18] +(34) HashAggregate [codegen id : 18] Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] Functions: [] Aggregate Attributes: [] Results [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -(43) RowToColumnar +(35) Exchange Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +Arguments: hashpartitioning(gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(44) CometColumnarExchange -Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -Arguments: hashpartitioning(gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(45) CometHashAggregate +(36) HashAggregate [codegen id : 19] Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] Functions: [] +Aggregate Attributes: [] +Results [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, CASE WHEN (t_class#21 = 0) THEN i_category#10 END AS _w0#62] -(46) CometColumnarExchange -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] -Arguments: hashpartitioning(lochierarchy#22, _w0#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(47) CometSort +(37) Exchange Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] -Arguments: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62], [lochierarchy#22 ASC NULLS FIRST, _w0#62 ASC NULLS FIRST, gross_margin#19 ASC NULLS FIRST] +Arguments: hashpartitioning(lochierarchy#22, _w0#62, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(48) ColumnarToRow [codegen id : 19] +(38) Sort [codegen id : 20] Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] +Arguments: [lochierarchy#22 ASC NULLS FIRST, _w0#62 ASC NULLS FIRST, gross_margin#19 ASC NULLS FIRST], false, 0 -(49) Window +(39) Window Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] Arguments: [rank(gross_margin#19) windowspecdefinition(lochierarchy#22, _w0#62, gross_margin#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#63], [lochierarchy#22, _w0#62], [gross_margin#19 ASC NULLS FIRST] -(50) Project [codegen id : 20] +(40) Project [codegen id : 21] Output [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] Input [6]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62, rank_within_parent#63] -(51) TakeOrderedAndProject +(41) TakeOrderedAndProject Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (56) -+- * ColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan parquet spark_catalog.default.date_dim (52) +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) (unknown) Scan parquet spark_catalog.default.date_dim @@ -299,18 +261,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(53) CometFilter +(43) CometFilter Input [2]: [d_date_sk#7, d_year#64] Condition : ((isnotnull(d_year#64) AND (d_year#64 = 2001)) AND isnotnull(d_date_sk#7)) -(54) CometProject +(44) CometProject Input [2]: [d_date_sk#7, d_year#64] Arguments: [d_date_sk#7], [d_date_sk#7] -(55) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(56) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index c7fb8c2303..f265d20995 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -1,84 +1,76 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (20) + WholeStageCodegen (21) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (19) - ColumnarToRow + WholeStageCodegen (20) + Sort [lochierarchy,_w0,gross_margin] InputAdapter - CometSort [lochierarchy,_w0,gross_margin] - CometColumnarExchange [lochierarchy,_w0] #1 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - RowToColumnar + Exchange [lochierarchy,_w0] #1 + WholeStageCodegen (19) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] [_w0] + InputAdapter + Exchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 WholeStageCodegen (18) HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] InputAdapter Union WholeStageCodegen (5) HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,t_category,t_class,lochierarchy,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #4 + InputAdapter + Exchange [i_category,i_class] #3 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (11) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 + InputAdapter + Exchange [i_category] #7 + WholeStageCodegen (10) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 WholeStageCodegen (17) HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange #8 - RowToColumnar - WholeStageCodegen (16) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 + InputAdapter + Exchange #8 + WholeStageCodegen (16) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index 109830f2ad..2be3c9c660 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -1,57 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (53) -+- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (32) - : : +- * Filter (31) - : : +- Window (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * ColumnarToRow (27) - : : +- CometSort (26) - : : +- CometColumnarExchange (25) - : : +- RowToColumnar (24) - : : +- * HashAggregate (23) - : : +- * ColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- RowToColumnar (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (42) - : +- * Project (41) - : +- Window (40) - : +- * ColumnarToRow (39) - : +- CometSort (38) - : +- CometColumnarExchange (37) - : +- RowToColumnar (36) - : +- * HashAggregate (35) - : +- * ColumnarToRow (34) - : +- ReusedExchange (33) - +- BroadcastExchange (50) - +- * Project (49) - +- Window (48) - +- * ColumnarToRow (47) - +- CometSort (46) - +- ReusedExchange (45) +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * Filter (25) + : : +- Window (24) + : : +- * Sort (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- Exchange (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) (unknown) Scan parquet spark_catalog.default.item @@ -97,7 +89,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 57] +(10) ReusedExchange [Reuses operator id: 49] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -145,149 +137,125 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [1]: [sum#15] Results [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -(20) RowToColumnar +(20) Exchange Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometColumnarExchange -Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) ColumnarToRow [codegen id : 5] -Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] - -(23) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 5] Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#17] Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS _w0#19] -(24) RowToColumnar -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] - -(25) CometColumnarExchange +(22) Exchange Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(26) CometSort +(23) Sort [codegen id : 6] Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 -(27) ColumnarToRow [codegen id : 6] -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] - -(28) Window +(24) Window Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(29) Filter [codegen id : 7] +(25) Filter [codegen id : 7] Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(30) Window +(26) Window Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] -(31) Filter [codegen id : 22] +(27) Filter [codegen id : 22] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(32) Project [codegen id : 22] +(28) Project [codegen id : 22] Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -(33) ReusedExchange [Reuses operator id: 21] +(29) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(34) ColumnarToRow [codegen id : 12] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(35) HashAggregate [codegen id : 12] +(30) HashAggregate [codegen id : 12] Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] -(36) RowToColumnar -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] - -(37) CometColumnarExchange +(31) Exchange Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) CometSort +(32) Sort [codegen id : 13] Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 -(39) ColumnarToRow [codegen id : 13] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] - -(40) Window +(33) Window Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(41) Project [codegen id : 14] +(34) Project [codegen id : 14] Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] -(42) BroadcastExchange +(35) BroadcastExchange Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 22] +(36) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] +(37) Project [codegen id : 22] Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -(45) ReusedExchange [Reuses operator id: 37] +(38) ReusedExchange [Reuses operator id: 31] Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -(46) CometSort +(39) Sort [codegen id : 20] Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 -(47) ColumnarToRow [codegen id : 20] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] - -(48) Window +(40) Window Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(49) Project [codegen id : 21] +(41) Project [codegen id : 21] Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] -(50) BroadcastExchange +(42) BroadcastExchange Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 22] +(43) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] Join type: Inner Join condition: None -(52) Project [codegen id : 22] +(44) Project [codegen id : 22] Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -(53) TakeOrderedAndProject +(45) TakeOrderedAndProject Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (57) -+- * ColumnarToRow (56) - +- CometFilter (55) - +- CometScan parquet spark_catalog.default.date_dim (54) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim @@ -297,14 +265,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(55) CometFilter +(47) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(56) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(57) BroadcastExchange +(49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index 0ea08b4352..a548953052 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -13,52 +13,48 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (6) - ColumnarToRow + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 WholeStageCodegen (14) @@ -66,16 +62,13 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (13) - ColumnarToRow + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - RowToColumnar - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + Exchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 WholeStageCodegen (21) @@ -83,7 +76,6 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (20) - ColumnarToRow + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index f2e628b732..f8419179d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -1,92 +1,78 @@ == Physical Plan == -* ColumnarToRow (88) -+- CometTakeOrderedAndProject (87) - +- CometHashAggregate (86) - +- CometColumnarExchange (85) - +- RowToColumnar (84) - +- * HashAggregate (83) - +- Union (82) - :- * Project (27) - : +- * Filter (26) - : +- Window (25) - : +- * Sort (24) - : +- Window (23) - : +- * ColumnarToRow (22) - : +- CometSort (21) - : +- CometColumnarExchange (20) - : +- RowToColumnar (19) - : +- * HashAggregate (18) - : +- * ColumnarToRow (17) - : +- CometColumnarExchange (16) - : +- RowToColumnar (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.web_returns (5) - : +- ReusedExchange (11) - :- * Project (54) - : +- * Filter (53) - : +- Window (52) - : +- * Sort (51) - : +- Window (50) - : +- * ColumnarToRow (49) - : +- CometSort (48) - : +- CometColumnarExchange (47) - : +- RowToColumnar (46) - : +- * HashAggregate (45) - : +- * ColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- RowToColumnar (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * ColumnarToRow (37) - : : +- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometBroadcastExchange (31) - : : : +- CometProject (30) - : : : +- CometFilter (29) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (28) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometScan parquet spark_catalog.default.catalog_returns (32) - : +- ReusedExchange (38) - +- * Project (81) - +- * Filter (80) - +- Window (79) - +- * Sort (78) - +- Window (77) - +- * ColumnarToRow (76) - +- CometSort (75) - +- CometColumnarExchange (74) - +- RowToColumnar (73) - +- * HashAggregate (72) - +- * ColumnarToRow (71) - +- CometColumnarExchange (70) - +- RowToColumnar (69) - +- * HashAggregate (68) - +- * Project (67) - +- * BroadcastHashJoin Inner BuildRight (66) - :- * ColumnarToRow (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometBroadcastExchange (58) - : : +- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan parquet spark_catalog.default.store_sales (55) - : +- CometProject (61) - : +- CometFilter (60) - : +- CometScan parquet spark_catalog.default.store_returns (59) - +- ReusedExchange (65) +TakeOrderedAndProject (74) ++- * HashAggregate (73) + +- Exchange (72) + +- * HashAggregate (71) + +- Union (70) + :- * Project (23) + : +- * Filter (22) + : +- Window (21) + : +- * Sort (20) + : +- Window (19) + : +- * Sort (18) + : +- Exchange (17) + : +- * HashAggregate (16) + : +- Exchange (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- ReusedExchange (11) + :- * Project (46) + : +- * Filter (45) + : +- Window (44) + : +- * Sort (43) + : +- Window (42) + : +- * Sort (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * ColumnarToRow (33) + : : +- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.catalog_returns (28) + : +- ReusedExchange (34) + +- * Project (69) + +- * Filter (68) + +- Window (67) + +- * Sort (66) + +- Window (65) + +- * Sort (64) + +- Exchange (63) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * ColumnarToRow (56) + : +- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometBroadcastExchange (50) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan parquet spark_catalog.default.store_sales (47) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.store_returns (51) + +- ReusedExchange (57) (unknown) Scan parquet spark_catalog.default.web_sales @@ -136,7 +122,7 @@ Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_re (10) ColumnarToRow [codegen id : 2] Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(11) ReusedExchange [Reuses operator id: 93] +(11) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#13] (12) BroadcastHashJoin [codegen id : 2] @@ -156,54 +142,42 @@ Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coa Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(15) RowToColumnar +(15) Exchange Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(16) CometColumnarExchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(17) ColumnarToRow [codegen id : 3] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(18) HashAggregate [codegen id : 3] +(16) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] -(19) RowToColumnar +(17) Exchange Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(20) CometColumnarExchange +(18) Sort [codegen id : 4] Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 -(21) CometSort -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] - -(22) ColumnarToRow [codegen id : 4] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] - -(23) Window +(19) Window Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(24) Sort [codegen id : 5] +(20) Sort [codegen id : 5] Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 -(25) Window +(21) Window Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(26) Filter [codegen id : 6] +(22) Filter [codegen id : 6] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(27) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] @@ -215,15 +189,15 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(30) CometProject +(26) CometProject Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -(31) CometBroadcastExchange +(27) CometBroadcastExchange Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] @@ -234,94 +208,82 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(33) CometFilter +(29) CometFilter Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) -(34) CometProject +(30) CometProject Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -(35) CometBroadcastHashJoin +(31) CometBroadcastHashJoin Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner -(36) CometProject +(32) CometProject Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -(37) ColumnarToRow [codegen id : 8] +(33) ColumnarToRow [codegen id : 8] Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -(38) ReusedExchange [Reuses operator id: 93] +(34) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#48] -(39) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#41] Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(40) Project [codegen id : 8] +(36) Project [codegen id : 8] Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] -(41) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 8] Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] Keys [1]: [cs_item_sk#36] Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -(42) RowToColumnar -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] - -(43) CometColumnarExchange -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(44) ColumnarToRow [codegen id : 9] +(38) Exchange Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(45) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 9] Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] Keys [1]: [cs_item_sk#36] Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] -(46) RowToColumnar -Input [3]: [item#65, return_ratio#66, currency_ratio#67] - -(47) CometColumnarExchange -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(48) CometSort +(40) Exchange Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [item#65, return_ratio#66, currency_ratio#67], [return_ratio#66 ASC NULLS FIRST] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(49) ColumnarToRow [codegen id : 10] +(41) Sort [codegen id : 10] Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 -(50) Window +(42) Window Input [3]: [item#65, return_ratio#66, currency_ratio#67] Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] -(51) Sort [codegen id : 11] +(43) Sort [codegen id : 11] Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 -(52) Window +(44) Window Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] -(53) Filter [codegen id : 12] +(45) Filter [codegen id : 12] Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) -(54) Project [codegen id : 12] +(46) Project [codegen id : 12] Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] @@ -333,15 +295,15 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(56) CometFilter +(48) CometFilter Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) -(57) CometProject +(49) CometProject Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -(58) CometBroadcastExchange +(50) CometBroadcastExchange Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] @@ -352,133 +314,117 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(60) CometFilter +(52) CometFilter Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) -(61) CometProject +(53) CometProject Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -(62) CometBroadcastHashJoin +(54) CometBroadcastHashJoin Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner -(63) CometProject +(55) CometProject Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -(64) ColumnarToRow [codegen id : 14] +(56) ColumnarToRow [codegen id : 14] Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -(65) ReusedExchange [Reuses operator id: 93] +(57) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#83] -(66) BroadcastHashJoin [codegen id : 14] +(58) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_sold_date_sk#76] Right keys [1]: [d_date_sk#83] Join type: Inner Join condition: None -(67) Project [codegen id : 14] +(59) Project [codegen id : 14] Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] -(68) HashAggregate [codegen id : 14] +(60) HashAggregate [codegen id : 14] Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] Keys [1]: [ss_item_sk#71] Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -(69) RowToColumnar -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(70) CometColumnarExchange +(61) Exchange Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(71) ColumnarToRow [codegen id : 15] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(72) HashAggregate [codegen id : 15] +(62) HashAggregate [codegen id : 15] Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] Keys [1]: [ss_item_sk#71] Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] -(73) RowToColumnar -Input [3]: [item#100, return_ratio#101, currency_ratio#102] - -(74) CometColumnarExchange +(63) Exchange Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(75) CometSort +(64) Sort [codegen id : 16] Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [item#100, return_ratio#101, currency_ratio#102], [return_ratio#101 ASC NULLS FIRST] +Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 -(76) ColumnarToRow [codegen id : 16] -Input [3]: [item#100, return_ratio#101, currency_ratio#102] - -(77) Window +(65) Window Input [3]: [item#100, return_ratio#101, currency_ratio#102] Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] -(78) Sort [codegen id : 17] +(66) Sort [codegen id : 17] Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 -(79) Window +(67) Window Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] -(80) Filter [codegen id : 18] +(68) Filter [codegen id : 18] Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) -(81) Project [codegen id : 18] +(69) Project [codegen id : 18] Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -(82) Union +(70) Union -(83) HashAggregate [codegen id : 19] +(71) HashAggregate [codegen id : 19] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(84) RowToColumnar -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(85) CometColumnarExchange +(72) Exchange Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(86) CometHashAggregate +(73) HashAggregate [codegen id : 20] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(87) CometTakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(88) ColumnarToRow [codegen id : 20] +(74) TakeOrderedAndProject Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (93) -+- * ColumnarToRow (92) - +- CometProject (91) - +- CometFilter (90) - +- CometScan parquet spark_catalog.default.date_dim (89) +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan parquet spark_catalog.default.date_dim (75) (unknown) Scan parquet spark_catalog.default.date_dim @@ -488,23 +434,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(90) CometFilter +(76) CometFilter Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) -(91) CometProject +(77) CometProject Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Arguments: [d_date_sk#13], [d_date_sk#13] -(92) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(93) BroadcastExchange +(79) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index 42f84a6e7f..43ebf34cc0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -1,135 +1,121 @@ -WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - RowToColumnar - WholeStageCodegen (19) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometSort [return_ratio] - CometColumnarExchange #2 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (12) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (11) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometSort [return_ratio] - CometColumnarExchange #6 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #8 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (18) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (17) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometSort [return_ratio] - CometColumnarExchange #9 - RowToColumnar - WholeStageCodegen (15) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - RowToColumnar - WholeStageCodegen (14) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #11 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 +TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] + WholeStageCodegen (20) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (19) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + Sort [return_ratio] + InputAdapter + Exchange #2 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (12) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (11) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (10) + Sort [return_ratio] + InputAdapter + Exchange #6 + WholeStageCodegen (9) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #7 + WholeStageCodegen (8) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #8 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (18) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (17) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (16) + Sort [return_ratio] + InputAdapter + Exchange #9 + WholeStageCodegen (15) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ss_item_sk] #10 + WholeStageCodegen (14) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #11 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index 3498539ba1..0dedd04721 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -1,89 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (85) -+- * Filter (84) - +- * HashAggregate (83) - +- * HashAggregate (82) - +- * Project (81) - +- * BroadcastHashJoin Inner BuildRight (80) - :- Window (73) - : +- * ColumnarToRow (72) - : +- CometSort (71) - : +- CometColumnarExchange (70) - : +- CometProject (69) - : +- CometFilter (68) - : +- CometSortMergeJoin (67) - : :- CometSort (33) - : : +- CometColumnarExchange (32) - : : +- RowToColumnar (31) - : : +- * HashAggregate (30) - : : +- * ColumnarToRow (29) - : : +- CometColumnarExchange (28) - : : +- RowToColumnar (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (17) - : : : +- Window (16) - : : : +- * ColumnarToRow (15) - : : : +- CometSort (14) - : : : +- CometColumnarExchange (13) - : : : +- RowToColumnar (12) - : : : +- * HashAggregate (11) - : : : +- * ColumnarToRow (10) - : : : +- CometColumnarExchange (9) - : : : +- RowToColumnar (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- Window (21) - : : +- * ColumnarToRow (20) - : : +- CometSort (19) - : : +- ReusedExchange (18) - : +- CometSort (66) - : +- CometColumnarExchange (65) - : +- RowToColumnar (64) - : +- * HashAggregate (63) - : +- * ColumnarToRow (62) - : +- CometColumnarExchange (61) - : +- RowToColumnar (60) - : +- * HashAggregate (59) - : +- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * Project (50) - : : +- Window (49) - : : +- * ColumnarToRow (48) - : : +- CometSort (47) - : : +- CometColumnarExchange (46) - : : +- RowToColumnar (45) - : : +- * HashAggregate (44) - : : +- * ColumnarToRow (43) - : : +- CometColumnarExchange (42) - : : +- RowToColumnar (41) - : : +- * HashAggregate (40) - : : +- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * ColumnarToRow (36) - : : : +- CometFilter (35) - : : : +- CometScan parquet spark_catalog.default.store_sales (34) - : : +- ReusedExchange (37) - : +- BroadcastExchange (56) - : +- * Project (55) - : +- Window (54) - : +- * ColumnarToRow (53) - : +- CometSort (52) - : +- ReusedExchange (51) - +- BroadcastExchange (79) - +- * Project (78) - +- Window (77) - +- * ColumnarToRow (76) - +- CometSort (75) - +- ReusedExchange (74) +TakeOrderedAndProject (67) ++- * Filter (66) + +- * HashAggregate (65) + +- * HashAggregate (64) + +- * Project (63) + +- * BroadcastHashJoin Inner BuildRight (62) + :- Window (56) + : +- * Sort (55) + : +- Exchange (54) + : +- * Project (53) + : +- * Filter (52) + : +- * SortMergeJoin FullOuter (51) + : :- * Sort (25) + : : +- Exchange (24) + : : +- * HashAggregate (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * Project (13) + : : : +- Window (12) + : : : +- * Sort (11) + : : : +- Exchange (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (18) + : : +- * Project (17) + : : +- Window (16) + : : +- * Sort (15) + : : +- ReusedExchange (14) + : +- * Sort (50) + : +- Exchange (49) + : +- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (38) + : : +- Window (37) + : : +- * Sort (36) + : : +- Exchange (35) + : : +- * HashAggregate (34) + : : +- Exchange (33) + : : +- * HashAggregate (32) + : : +- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * ColumnarToRow (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.store_sales (26) + : : +- ReusedExchange (29) + : +- BroadcastExchange (43) + : +- * Project (42) + : +- Window (41) + : +- * Sort (40) + : +- ReusedExchange (39) + +- BroadcastExchange (61) + +- * Project (60) + +- Window (59) + +- * Sort (58) + +- ReusedExchange (57) (unknown) Scan parquet spark_catalog.default.web_sales @@ -101,7 +83,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 2] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 90] +(4) ReusedExchange [Reuses operator id: 72] Output [2]: [d_date_sk#5, d_date#6] (5) BroadcastHashJoin [codegen id : 2] @@ -121,111 +103,87 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] Aggregate Attributes [1]: [sum#7] Results [3]: [ws_item_sk#1, d_date#6, sum#8] -(8) RowToColumnar +(8) Exchange Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) CometColumnarExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(10) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(11) HashAggregate [codegen id : 3] +(9) HashAggregate [codegen id : 3] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] -(12) RowToColumnar -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] - -(13) CometColumnarExchange +(10) Exchange Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(14) CometSort +(11) Sort [codegen id : 4] Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(15) ColumnarToRow [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] - -(16) Window +(12) Window Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(17) Project [codegen id : 10] +(13) Project [codegen id : 10] Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] -(18) ReusedExchange [Reuses operator id: 13] +(14) ReusedExchange [Reuses operator id: 10] Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -(19) CometSort -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14], [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] - -(20) ColumnarToRow [codegen id : 8] +(15) Sort [codegen id : 8] Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +Arguments: [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST], false, 0 -(21) Window +(16) Window Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] -(22) Project [codegen id : 9] +(17) Project [codegen id : 9] Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] -(23) BroadcastExchange +(18) BroadcastExchange Input [3]: [item_sk#16, sumws#17, rk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 10] +(19) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#10] Right keys [1]: [item_sk#16] Join type: Inner Join condition: (rk#12 >= rk#15) -(25) Project [codegen id : 10] +(20) Project [codegen id : 10] Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] -(26) HashAggregate [codegen id : 10] +(21) HashAggregate [codegen id : 10] Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [partial_sum(sumws#17)] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(27) RowToColumnar -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(28) CometColumnarExchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) ColumnarToRow [codegen id : 11] +(22) Exchange Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(30) HashAggregate [codegen id : 11] +(23) HashAggregate [codegen id : 11] Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [sum(sumws#17)] Aggregate Attributes [1]: [sum(sumws#17)#22] Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] -(31) RowToColumnar -Input [3]: [item_sk#10, d_date#6, cume_sales#23] - -(32) CometColumnarExchange +(24) Exchange Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(33) CometSort +(25) Sort [codegen id : 12] Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +Arguments: [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] @@ -235,229 +193,200 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(35) CometFilter +(27) CometFilter Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#24) -(36) ColumnarToRow [codegen id : 13] +(28) ColumnarToRow [codegen id : 14] Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -(37) ReusedExchange [Reuses operator id: 90] +(29) ReusedExchange [Reuses operator id: 72] Output [2]: [d_date_sk#28, d_date#29] -(38) BroadcastHashJoin [codegen id : 13] +(30) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#28] Join type: Inner Join condition: None -(39) Project [codegen id : 13] +(31) Project [codegen id : 14] Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] -(40) HashAggregate [codegen id : 13] +(32) HashAggregate [codegen id : 14] Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] Keys [2]: [ss_item_sk#24, d_date#29] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] Aggregate Attributes [1]: [sum#30] Results [3]: [ss_item_sk#24, d_date#29, sum#31] -(41) RowToColumnar -Input [3]: [ss_item_sk#24, d_date#29, sum#31] - -(42) CometColumnarExchange +(33) Exchange Input [3]: [ss_item_sk#24, d_date#29, sum#31] -Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(43) ColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#24, d_date#29, sum#31] - -(44) HashAggregate [codegen id : 14] +(34) HashAggregate [codegen id : 15] Input [3]: [ss_item_sk#24, d_date#29, sum#31] Keys [2]: [ss_item_sk#24, d_date#29] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#32] Results [4]: [ss_item_sk#24 AS item_sk#33, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#32,17,2) AS sumss#34, ss_item_sk#24] -(45) RowToColumnar -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] - -(46) CometColumnarExchange -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(47) CometSort +(35) Exchange Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] +Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(48) ColumnarToRow [codegen id : 15] +(36) Sort [codegen id : 16] Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 -(49) Window +(37) Window Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] -(50) Project [codegen id : 21] +(38) Project [codegen id : 22] Output [4]: [item_sk#33, d_date#29, sumss#34, rk#35] Input [5]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24, rk#35] -(51) ReusedExchange [Reuses operator id: 46] +(39) ReusedExchange [Reuses operator id: 35] Output [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -(52) CometSort -Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -Arguments: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37], [ss_item_sk#37 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] - -(53) ColumnarToRow [codegen id : 19] +(40) Sort [codegen id : 20] Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +Arguments: [ss_item_sk#37 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST], false, 0 -(54) Window +(41) Window Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] Arguments: [row_number() windowspecdefinition(ss_item_sk#37, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [ss_item_sk#37], [d_date#36 ASC NULLS FIRST] -(55) Project [codegen id : 20] +(42) Project [codegen id : 21] Output [3]: [item_sk#33 AS item_sk#39, sumss#34 AS sumss#40, rk#38] Input [5]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37, rk#38] -(56) BroadcastExchange +(43) BroadcastExchange Input [3]: [item_sk#39, sumss#40, rk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 21] +(44) BroadcastHashJoin [codegen id : 22] Left keys [1]: [item_sk#33] Right keys [1]: [item_sk#39] Join type: Inner Join condition: (rk#35 >= rk#38) -(58) Project [codegen id : 21] +(45) Project [codegen id : 22] Output [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] Input [7]: [item_sk#33, d_date#29, sumss#34, rk#35, item_sk#39, sumss#40, rk#38] -(59) HashAggregate [codegen id : 21] +(46) HashAggregate [codegen id : 22] Input [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] Keys [3]: [item_sk#33, d_date#29, sumss#34] Functions [1]: [partial_sum(sumss#40)] Aggregate Attributes [2]: [sum#41, isEmpty#42] Results [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -(60) RowToColumnar -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] - -(61) CometColumnarExchange +(47) Exchange Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -Arguments: hashpartitioning(item_sk#33, d_date#29, sumss#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Arguments: hashpartitioning(item_sk#33, d_date#29, sumss#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(62) ColumnarToRow [codegen id : 22] -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] - -(63) HashAggregate [codegen id : 22] +(48) HashAggregate [codegen id : 23] Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] Keys [3]: [item_sk#33, d_date#29, sumss#34] Functions [1]: [sum(sumss#40)] Aggregate Attributes [1]: [sum(sumss#40)#45] Results [3]: [item_sk#33, d_date#29, sum(sumss#40)#45 AS cume_sales#46] -(64) RowToColumnar +(49) Exchange Input [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: hashpartitioning(item_sk#33, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(65) CometColumnarExchange +(50) Sort [codegen id : 24] Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: hashpartitioning(item_sk#33, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Arguments: [item_sk#33 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 -(66) CometSort -Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#33, d_date#29, cume_sales#46], [item_sk#33 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] - -(67) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#23] -Right output [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#10, d_date#6], [item_sk#33, d_date#29], FullOuter +(51) SortMergeJoin [codegen id : 25] +Left keys [2]: [item_sk#10, d_date#6] +Right keys [2]: [item_sk#33, d_date#29] +Join type: FullOuter +Join condition: None -(68) CometFilter +(52) Filter [codegen id : 25] Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END) -(69) CometProject +(53) Project [codegen id : 25] +Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END AS item_sk#47, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#46 AS store_sales#50] Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END AS item_sk#47, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#46 AS store_sales#50] -(70) CometColumnarExchange +(54) Exchange Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: hashpartitioning(item_sk#47, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +Arguments: hashpartitioning(item_sk#47, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(71) CometSort +(55) Sort [codegen id : 26] Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] +Arguments: [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], false, 0 -(72) ColumnarToRow [codegen id : 23] -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] - -(73) Window +(56) Window Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#47], [d_date#48 ASC NULLS FIRST] -(74) ReusedExchange [Reuses operator id: 70] +(57) ReusedExchange [Reuses operator id: 54] Output [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -(75) CometSort -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] - -(76) ColumnarToRow [codegen id : 46] +(58) Sort [codegen id : 52] Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], false, 0 -(77) Window +(59) Window Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#52], [item_sk#47], [d_date#48 ASC NULLS FIRST] -(78) Project [codegen id : 47] +(60) Project [codegen id : 53] Output [4]: [item_sk#47 AS item_sk#53, web_sales#49 AS web_sales#54, store_sales#50 AS store_sales#55, rk#52] Input [5]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#52] -(79) BroadcastExchange +(61) BroadcastExchange Input [4]: [item_sk#53, web_sales#54, store_sales#55, rk#52] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(80) BroadcastHashJoin [codegen id : 48] +(62) BroadcastHashJoin [codegen id : 54] Left keys [1]: [item_sk#47] Right keys [1]: [item_sk#53] Join type: Inner Join condition: (rk#51 >= rk#52) -(81) Project [codegen id : 48] +(63) Project [codegen id : 54] Output [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] Input [9]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#51, item_sk#53, web_sales#54, store_sales#55, rk#52] -(82) HashAggregate [codegen id : 48] +(64) HashAggregate [codegen id : 54] Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] Functions [2]: [partial_max(web_sales#54), partial_max(store_sales#55)] Aggregate Attributes [2]: [max#56, max#57] Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] -(83) HashAggregate [codegen id : 48] +(65) HashAggregate [codegen id : 54] Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] Functions [2]: [max(web_sales#54), max(store_sales#55)] Aggregate Attributes [2]: [max(web_sales#54)#60, max(store_sales#55)#61] Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max(web_sales#54)#60 AS web_cumulative#62, max(store_sales#55)#61 AS store_cumulative#63] -(84) Filter [codegen id : 48] +(66) Filter [codegen id : 54] Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) -(85) TakeOrderedAndProject +(67) TakeOrderedAndProject Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] Arguments: 100, [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (90) -+- * ColumnarToRow (89) - +- CometProject (88) - +- CometFilter (87) - +- CometScan parquet spark_catalog.default.date_dim (86) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) (unknown) Scan parquet spark_catalog.default.date_dim @@ -467,21 +396,21 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter +(69) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) -(88) CometProject +(70) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(89) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(90) BroadcastExchange +(72) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 34 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index 3fa0c2c698..3109290dc7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (48) + WholeStageCodegen (54) Filter [web_cumulative,store_cumulative] HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] @@ -7,23 +7,23 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (23) - ColumnarToRow + WholeStageCodegen (26) + Sort [item_sk,d_date] InputAdapter - CometSort [item_sk,d_date] - CometColumnarExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,item_sk] - CometSortMergeJoin [item_sk,d_date,item_sk,d_date] - CometSort [item_sk,d_date] - CometColumnarExchange [item_sk,d_date] #2 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - RowToColumnar + Exchange [item_sk] #1 + WholeStageCodegen (25) + Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] + Filter [item_sk,item_sk] + SortMergeJoin [item_sk,d_date,item_sk,d_date] + InputAdapter + WholeStageCodegen (12) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #2 + WholeStageCodegen (11) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + InputAdapter + Exchange [item_sk,d_date,sumws] #3 WholeStageCodegen (10) HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] Project [item_sk,d_date,sumws,sumws] @@ -32,35 +32,31 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ws_item_sk,d_date] WholeStageCodegen (4) - ColumnarToRow + Sort [ws_item_sk,d_date] InputAdapter - CometSort [ws_item_sk,d_date] - CometColumnarExchange [ws_item_sk] #4 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #5 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + Exchange [ws_item_sk] #4 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + InputAdapter + Exchange [ws_item_sk,d_date] #5 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 InputAdapter BroadcastExchange #7 WholeStageCodegen (9) @@ -68,68 +64,61 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store InputAdapter Window [ws_item_sk,d_date] WholeStageCodegen (8) - ColumnarToRow + Sort [ws_item_sk,d_date] InputAdapter - CometSort [ws_item_sk,d_date] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date] - CometColumnarExchange [item_sk,d_date] #8 - RowToColumnar - WholeStageCodegen (22) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #9 - RowToColumnar - WholeStageCodegen (21) + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + InputAdapter + WholeStageCodegen (24) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #8 + WholeStageCodegen (23) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + InputAdapter + Exchange [item_sk,d_date,sumss] #9 + WholeStageCodegen (22) HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] Project [item_sk,d_date,sumss,sumss] BroadcastHashJoin [item_sk,item_sk,rk,rk] Project [item_sk,d_date,sumss,rk] InputAdapter Window [ss_item_sk,d_date] - WholeStageCodegen (15) - ColumnarToRow + WholeStageCodegen (16) + Sort [ss_item_sk,d_date] InputAdapter - CometSort [ss_item_sk,d_date] - CometColumnarExchange [ss_item_sk] #10 - RowToColumnar - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #11 - RowToColumnar - WholeStageCodegen (13) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + Exchange [ss_item_sk] #10 + WholeStageCodegen (15) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + InputAdapter + Exchange [ss_item_sk,d_date] #11 + WholeStageCodegen (14) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 InputAdapter BroadcastExchange #12 - WholeStageCodegen (20) + WholeStageCodegen (21) Project [item_sk,sumss,rk] InputAdapter Window [ss_item_sk,d_date] - WholeStageCodegen (19) - ColumnarToRow + WholeStageCodegen (20) + Sort [ss_item_sk,d_date] InputAdapter - CometSort [ss_item_sk,d_date] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 InputAdapter BroadcastExchange #13 - WholeStageCodegen (47) + WholeStageCodegen (53) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (46) - ColumnarToRow + WholeStageCodegen (52) + Sort [item_sk,d_date] InputAdapter - CometSort [item_sk,d_date] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index f935c102e7..dff12158b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -1,57 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (53) -+- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (32) - : : +- * Filter (31) - : : +- Window (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * ColumnarToRow (27) - : : +- CometSort (26) - : : +- CometColumnarExchange (25) - : : +- RowToColumnar (24) - : : +- * HashAggregate (23) - : : +- * ColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- RowToColumnar (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (42) - : +- * Project (41) - : +- Window (40) - : +- * ColumnarToRow (39) - : +- CometSort (38) - : +- CometColumnarExchange (37) - : +- RowToColumnar (36) - : +- * HashAggregate (35) - : +- * ColumnarToRow (34) - : +- ReusedExchange (33) - +- BroadcastExchange (50) - +- * Project (49) - +- Window (48) - +- * ColumnarToRow (47) - +- CometSort (46) - +- ReusedExchange (45) +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * Filter (25) + : : +- Window (24) + : : +- * Sort (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- Exchange (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.call_center (13) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) (unknown) Scan parquet spark_catalog.default.item @@ -97,7 +89,7 @@ Join condition: None Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 57] +(10) ReusedExchange [Reuses operator id: 49] Output [3]: [d_date_sk#9, d_year#10, d_moy#11] (11) BroadcastHashJoin [codegen id : 4] @@ -145,149 +137,125 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#6))] Aggregate Attributes [1]: [sum#14] Results [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -(20) RowToColumnar +(20) Exchange Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometColumnarExchange -Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) ColumnarToRow [codegen id : 5] -Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] - -(23) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 5] Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#16] Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS _w0#18] -(24) RowToColumnar -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] - -(25) CometColumnarExchange +(22) Exchange Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(26) CometSort +(23) Sort [codegen id : 6] Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18], [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 -(27) ColumnarToRow [codegen id : 6] -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] - -(28) Window +(24) Window Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(29) Filter [codegen id : 7] +(25) Filter [codegen id : 7] Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) -(30) Window +(26) Window Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] -(31) Filter [codegen id : 22] +(27) Filter [codegen id : 22] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(32) Project [codegen id : 22] +(28) Project [codegen id : 22] Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -(33) ReusedExchange [Reuses operator id: 21] +(29) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(34) ColumnarToRow [codegen id : 12] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] - -(35) HashAggregate [codegen id : 12] +(30) HashAggregate [codegen id : 12] Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] -(36) RowToColumnar -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] - -(37) CometColumnarExchange +(31) Exchange Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) CometSort +(32) Sort [codegen id : 13] Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 -(39) ColumnarToRow [codegen id : 13] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] - -(40) Window +(33) Window Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(41) Project [codegen id : 14] +(34) Project [codegen id : 14] Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] -(42) BroadcastExchange +(35) BroadcastExchange Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 22] +(36) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] +(37) Project [codegen id : 22] Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -(45) ReusedExchange [Reuses operator id: 37] +(38) ReusedExchange [Reuses operator id: 31] Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -(46) CometSort +(39) Sort [codegen id : 20] Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 -(47) ColumnarToRow [codegen id : 20] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] - -(48) Window +(40) Window Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(49) Project [codegen id : 21] +(41) Project [codegen id : 21] Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] -(50) BroadcastExchange +(42) BroadcastExchange Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 22] +(43) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(52) Project [codegen id : 22] +(44) Project [codegen id : 22] Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -(53) TakeOrderedAndProject +(45) TakeOrderedAndProject Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (57) -+- * ColumnarToRow (56) - +- CometFilter (55) - +- CometScan parquet spark_catalog.default.date_dim (54) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) (unknown) Scan parquet spark_catalog.default.date_dim @@ -297,14 +265,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(55) CometFilter +(47) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(56) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(57) BroadcastExchange +(49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index a591ad8c2c..56e33be9e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -13,52 +13,48 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (6) - ColumnarToRow + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + Exchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #3 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometFilter [cs_item_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 WholeStageCodegen (14) @@ -66,16 +62,13 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (13) - ColumnarToRow + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - RowToColumnar - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + Exchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 WholeStageCodegen (21) @@ -83,7 +76,6 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (20) - ColumnarToRow + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 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 d2a6deea76..d0c446ece2 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,167 +1,141 @@ == Physical Plan == -* ColumnarToRow (163) -+- CometTakeOrderedAndProject (162) - +- CometHashAggregate (161) - +- CometColumnarExchange (160) - +- RowToColumnar (159) - +- * HashAggregate (158) - +- Union (157) - :- * HashAggregate (76) - : +- * ColumnarToRow (75) - : +- CometColumnarExchange (74) - : +- RowToColumnar (73) - : +- * HashAggregate (72) - : +- Union (71) - : :- * HashAggregate (22) - : : +- * ColumnarToRow (21) - : : +- CometColumnarExchange (20) - : : +- RowToColumnar (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (11) - : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : :- * ColumnarToRow (8) - : : : : +- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : +- ReusedExchange (9) - : : +- BroadcastExchange (15) - : : +- * ColumnarToRow (14) - : : +- CometFilter (13) - : : +- CometScan parquet spark_catalog.default.store (12) - : :- * HashAggregate (44) - : : +- * ColumnarToRow (43) - : : +- CometColumnarExchange (42) - : : +- RowToColumnar (41) - : : +- * HashAggregate (40) - : : +- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (33) - : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : :- * ColumnarToRow (30) - : : : : +- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (31) - : : +- BroadcastExchange (37) - : : +- * ColumnarToRow (36) - : : +- CometFilter (35) - : : +- CometScan parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (70) - : +- * ColumnarToRow (69) - : +- CometColumnarExchange (68) - : +- RowToColumnar (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * ColumnarToRow (56) - : : : +- CometUnion (55) - : : : :- CometProject (47) - : : : : +- CometFilter (46) - : : : : +- CometScan parquet spark_catalog.default.web_sales (45) - : : : +- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometBroadcastExchange (49) - : : : : +- CometScan parquet spark_catalog.default.web_returns (48) - : : : +- CometProject (52) - : : : +- CometFilter (51) - : : : +- CometScan parquet spark_catalog.default.web_sales (50) - : : +- ReusedExchange (57) - : +- BroadcastExchange (63) - : +- * ColumnarToRow (62) - : +- CometFilter (61) - : +- CometScan parquet spark_catalog.default.web_site (60) - :- * HashAggregate (116) - : +- * ColumnarToRow (115) - : +- CometColumnarExchange (114) - : +- RowToColumnar (113) - : +- * HashAggregate (112) - : +- * HashAggregate (111) - : +- * ColumnarToRow (110) - : +- CometColumnarExchange (109) - : +- RowToColumnar (108) - : +- * HashAggregate (107) - : +- Union (106) - : :- * HashAggregate (79) - : : +- * ColumnarToRow (78) - : : +- ReusedExchange (77) - : :- * HashAggregate (82) - : : +- * ColumnarToRow (81) - : : +- ReusedExchange (80) - : +- * HashAggregate (105) - : +- * ColumnarToRow (104) - : +- CometColumnarExchange (103) - : +- RowToColumnar (102) - : +- * HashAggregate (101) - : +- * Project (100) - : +- * BroadcastHashJoin Inner BuildRight (99) - : :- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * ColumnarToRow (94) - : : : +- CometUnion (93) - : : : :- CometProject (85) - : : : : +- CometFilter (84) - : : : : +- CometScan parquet spark_catalog.default.web_sales (83) - : : : +- CometProject (92) - : : : +- CometBroadcastHashJoin (91) - : : : :- CometBroadcastExchange (87) - : : : : +- CometScan parquet spark_catalog.default.web_returns (86) - : : : +- CometProject (90) - : : : +- CometFilter (89) - : : : +- CometScan parquet spark_catalog.default.web_sales (88) - : : +- ReusedExchange (95) - : +- ReusedExchange (98) - +- * HashAggregate (156) - +- * ColumnarToRow (155) - +- CometColumnarExchange (154) - +- RowToColumnar (153) - +- * HashAggregate (152) - +- * HashAggregate (151) - +- * ColumnarToRow (150) - +- CometColumnarExchange (149) - +- RowToColumnar (148) - +- * HashAggregate (147) - +- Union (146) - :- * HashAggregate (119) - : +- * ColumnarToRow (118) - : +- ReusedExchange (117) - :- * HashAggregate (122) - : +- * ColumnarToRow (121) - : +- ReusedExchange (120) - +- * HashAggregate (145) - +- * ColumnarToRow (144) - +- CometColumnarExchange (143) - +- RowToColumnar (142) - +- * HashAggregate (141) - +- * Project (140) - +- * BroadcastHashJoin Inner BuildRight (139) - :- * Project (137) - : +- * BroadcastHashJoin Inner BuildRight (136) - : :- * ColumnarToRow (134) - : : +- CometUnion (133) - : : :- CometProject (125) - : : : +- CometFilter (124) - : : : +- CometScan parquet spark_catalog.default.web_sales (123) - : : +- CometProject (132) - : : +- CometBroadcastHashJoin (131) - : : :- CometBroadcastExchange (127) - : : : +- CometScan parquet spark_catalog.default.web_returns (126) - : : +- CometProject (130) - : : +- CometFilter (129) - : : +- CometScan parquet spark_catalog.default.web_sales (128) - : +- ReusedExchange (135) - +- ReusedExchange (138) +TakeOrderedAndProject (137) ++- * HashAggregate (136) + +- Exchange (135) + +- * HashAggregate (134) + +- Union (133) + :- * HashAggregate (68) + : +- Exchange (67) + : +- * HashAggregate (66) + : +- Union (65) + : :- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Project (11) + : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : :- * ColumnarToRow (8) + : : : : +- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : +- ReusedExchange (9) + : : +- BroadcastExchange (15) + : : +- * ColumnarToRow (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.store (12) + : :- * HashAggregate (40) + : : +- Exchange (39) + : : +- * HashAggregate (38) + : : +- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (31) + : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : :- * ColumnarToRow (28) + : : : : +- CometUnion (27) + : : : : :- CometProject (23) + : : : : : +- CometFilter (22) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) + : : : : +- CometProject (26) + : : : : +- CometFilter (25) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (24) + : : : +- ReusedExchange (29) + : : +- BroadcastExchange (35) + : : +- * ColumnarToRow (34) + : : +- CometFilter (33) + : : +- CometScan parquet spark_catalog.default.catalog_page (32) + : +- * HashAggregate (64) + : +- Exchange (63) + : +- * HashAggregate (62) + : +- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * Project (55) + : : +- * BroadcastHashJoin Inner BuildRight (54) + : : :- * ColumnarToRow (52) + : : : +- CometUnion (51) + : : : :- CometProject (43) + : : : : +- CometFilter (42) + : : : : +- CometScan parquet spark_catalog.default.web_sales (41) + : : : +- CometProject (50) + : : : +- CometBroadcastHashJoin (49) + : : : :- CometBroadcastExchange (45) + : : : : +- CometScan parquet spark_catalog.default.web_returns (44) + : : : +- CometProject (48) + : : : +- CometFilter (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- ReusedExchange (53) + : +- BroadcastExchange (59) + : +- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_site (56) + :- * HashAggregate (100) + : +- Exchange (99) + : +- * HashAggregate (98) + : +- * HashAggregate (97) + : +- Exchange (96) + : +- * HashAggregate (95) + : +- Union (94) + : :- * HashAggregate (70) + : : +- ReusedExchange (69) + : :- * HashAggregate (72) + : : +- ReusedExchange (71) + : +- * HashAggregate (93) + : +- Exchange (92) + : +- * HashAggregate (91) + : +- * Project (90) + : +- * BroadcastHashJoin Inner BuildRight (89) + : :- * Project (87) + : : +- * BroadcastHashJoin Inner BuildRight (86) + : : :- * ColumnarToRow (84) + : : : +- CometUnion (83) + : : : :- CometProject (75) + : : : : +- CometFilter (74) + : : : : +- CometScan parquet spark_catalog.default.web_sales (73) + : : : +- CometProject (82) + : : : +- CometBroadcastHashJoin (81) + : : : :- CometBroadcastExchange (77) + : : : : +- CometScan parquet spark_catalog.default.web_returns (76) + : : : +- CometProject (80) + : : : +- CometFilter (79) + : : : +- CometScan parquet spark_catalog.default.web_sales (78) + : : +- ReusedExchange (85) + : +- ReusedExchange (88) + +- * HashAggregate (132) + +- Exchange (131) + +- * HashAggregate (130) + +- * HashAggregate (129) + +- Exchange (128) + +- * HashAggregate (127) + +- Union (126) + :- * HashAggregate (102) + : +- ReusedExchange (101) + :- * HashAggregate (104) + : +- ReusedExchange (103) + +- * HashAggregate (125) + +- Exchange (124) + +- * HashAggregate (123) + +- * Project (122) + +- * BroadcastHashJoin Inner BuildRight (121) + :- * Project (119) + : +- * BroadcastHashJoin Inner BuildRight (118) + : :- * ColumnarToRow (116) + : : +- CometUnion (115) + : : :- CometProject (107) + : : : +- CometFilter (106) + : : : +- CometScan parquet spark_catalog.default.web_sales (105) + : : +- CometProject (114) + : : +- CometBroadcastHashJoin (113) + : : :- CometBroadcastExchange (109) + : : : +- CometScan parquet spark_catalog.default.web_returns (108) + : : +- CometProject (112) + : : +- CometFilter (111) + : : +- CometScan parquet spark_catalog.default.web_sales (110) + : +- ReusedExchange (117) + +- ReusedExchange (120) (unknown) Scan parquet spark_catalog.default.store_sales @@ -203,7 +177,7 @@ Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_a (8) ColumnarToRow [codegen id : 3] Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -(9) ReusedExchange [Reuses operator id: 168] +(9) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#22] (10) BroadcastHashJoin [codegen id : 3] @@ -251,17 +225,11 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -(19) RowToColumnar +(19) Exchange Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(20) CometColumnarExchange -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(21) ColumnarToRow [codegen id : 4] -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] - -(22) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 4] Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] Keys [1]: [s_store_id#24] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] @@ -276,11 +244,11 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(24) CometFilter +(22) CometFilter Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Condition : isnotnull(cs_catalog_page_sk#42) -(25) CometProject +(23) CometProject Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Arguments: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52], [cs_catalog_page_sk#42 AS page_sk#47, cs_sold_date_sk#45 AS date_sk#48, cs_ext_sales_price#43 AS sales_price#49, cs_net_profit#44 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] @@ -292,31 +260,31 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(27) CometFilter +(25) CometFilter Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Condition : isnotnull(cr_catalog_page_sk#53) -(28) CometProject +(26) CometProject Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Arguments: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62], [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] -(29) CometUnion +(27) CometUnion Child 0 Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62] -(30) ColumnarToRow [codegen id : 7] +(28) ColumnarToRow [codegen id : 7] Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] -(31) ReusedExchange [Reuses operator id: 168] +(29) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#63] -(32) BroadcastHashJoin [codegen id : 7] +(30) BroadcastHashJoin [codegen id : 7] Left keys [1]: [date_sk#48] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(31) Project [codegen id : 7] Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] @@ -327,45 +295,39 @@ Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter +(33) CometFilter Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Condition : isnotnull(cp_catalog_page_sk#64) -(36) ColumnarToRow [codegen id : 6] +(34) ColumnarToRow [codegen id : 6] Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -(37) BroadcastExchange +(35) BroadcastExchange Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 7] +(36) BroadcastHashJoin [codegen id : 7] Left keys [1]: [page_sk#47] Right keys [1]: [cp_catalog_page_sk#64] Join type: Inner Join condition: None -(39) Project [codegen id : 7] +(37) Project [codegen id : 7] Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#65] -(40) HashAggregate [codegen id : 7] +(38) HashAggregate [codegen id : 7] Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] Aggregate Attributes [4]: [sum#66, sum#67, sum#68, sum#69] Results [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -(41) RowToColumnar -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] - -(42) CometColumnarExchange -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(43) ColumnarToRow [codegen id : 8] +(39) Exchange Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(44) HashAggregate [codegen id : 8] +(40) HashAggregate [codegen id : 8] Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -380,11 +342,11 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(46) CometFilter +(42) CometFilter Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Condition : isnotnull(ws_web_site_sk#83) -(47) CometProject +(43) CometProject Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] @@ -395,7 +357,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] ReadSchema: struct -(49) CometBroadcastExchange +(45) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] @@ -406,40 +368,40 @@ 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 -(51) CometFilter +(47) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(52) CometProject +(48) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(53) CometBroadcastHashJoin +(49) CometBroadcastHashJoin Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(54) CometProject +(50) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -(55) CometUnion +(51) CometUnion Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] -(56) ColumnarToRow [codegen id : 11] +(52) ColumnarToRow [codegen id : 11] Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(57) ReusedExchange [Reuses operator id: 168] +(53) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#109] -(58) BroadcastHashJoin [codegen id : 11] +(54) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(59) Project [codegen id : 11] +(55) Project [codegen id : 11] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -450,97 +412,79 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(61) CometFilter +(57) CometFilter Input [2]: [web_site_sk#110, web_site_id#111] Condition : isnotnull(web_site_sk#110) -(62) ColumnarToRow [codegen id : 10] +(58) ColumnarToRow [codegen id : 10] Input [2]: [web_site_sk#110, web_site_id#111] -(63) BroadcastExchange +(59) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#111] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(64) BroadcastHashJoin [codegen id : 11] +(60) BroadcastHashJoin [codegen id : 11] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(65) Project [codegen id : 11] +(61) Project [codegen id : 11] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(66) HashAggregate [codegen id : 11] +(62) HashAggregate [codegen id : 11] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -(67) RowToColumnar -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] - -(68) CometColumnarExchange +(63) Exchange Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(69) ColumnarToRow [codegen id : 12] -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] - -(70) HashAggregate [codegen id : 12] +(64) HashAggregate [codegen id : 12] Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] -(71) Union +(65) Union -(72) HashAggregate [codegen id : 13] +(66) HashAggregate [codegen id : 13] Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] Keys [2]: [channel#37, id#38] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -(73) RowToColumnar -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] - -(74) CometColumnarExchange +(67) Exchange Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(75) ColumnarToRow [codegen id : 14] -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] - -(76) HashAggregate [codegen id : 14] +(68) HashAggregate [codegen id : 14] Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] Results [5]: [channel#37, id#38, cast(sum(sales#39)#141 as decimal(37,2)) AS sales#144, cast(sum(returns#40)#142 as decimal(37,2)) AS returns#145, cast(sum(profit#41)#143 as decimal(38,2)) AS profit#146] -(77) ReusedExchange [Reuses operator id: 20] +(69) ReusedExchange [Reuses operator id: 19] Output [5]: [s_store_id#24, sum#147, sum#148, sum#149, sum#150] -(78) ColumnarToRow [codegen id : 18] -Input [5]: [s_store_id#24, sum#147, sum#148, sum#149, sum#150] - -(79) HashAggregate [codegen id : 18] +(70) HashAggregate [codegen id : 18] Input [5]: [s_store_id#24, sum#147, sum#148, sum#149, sum#150] Keys [1]: [s_store_id#24] 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))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] -(80) ReusedExchange [Reuses operator id: 42] +(71) ReusedExchange [Reuses operator id: 39] Output [5]: [cp_catalog_page_id#65, sum#151, sum#152, sum#153, sum#154] -(81) ColumnarToRow [codegen id : 22] -Input [5]: [cp_catalog_page_id#65, sum#151, sum#152, sum#153, sum#154] - -(82) HashAggregate [codegen id : 22] +(72) HashAggregate [codegen id : 22] Input [5]: [cp_catalog_page_id#65, sum#151, sum#152, sum#153, sum#154] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -555,11 +499,11 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(84) CometFilter +(74) CometFilter Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Condition : isnotnull(ws_web_site_sk#83) -(85) CometProject +(75) CometProject Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] @@ -570,7 +514,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#155)] ReadSchema: struct -(87) CometBroadcastExchange +(77) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] @@ -581,150 +525,126 @@ 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 -(89) CometFilter +(79) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(90) CometProject +(80) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(91) CometBroadcastHashJoin +(81) CometBroadcastHashJoin Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(92) CometProject +(82) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -(93) CometUnion +(83) CometUnion Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] -(94) ColumnarToRow [codegen id : 25] +(84) ColumnarToRow [codegen id : 25] Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(95) ReusedExchange [Reuses operator id: 168] +(85) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#109] -(96) BroadcastHashJoin [codegen id : 25] +(86) BroadcastHashJoin [codegen id : 25] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(97) Project [codegen id : 25] +(87) Project [codegen id : 25] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] -(98) ReusedExchange [Reuses operator id: 63] +(88) ReusedExchange [Reuses operator id: 59] Output [2]: [web_site_sk#110, web_site_id#111] -(99) BroadcastHashJoin [codegen id : 25] +(89) BroadcastHashJoin [codegen id : 25] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(100) Project [codegen id : 25] +(90) Project [codegen id : 25] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(101) HashAggregate [codegen id : 25] +(91) HashAggregate [codegen id : 25] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum#156, sum#157, sum#158, sum#159] Results [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] -(102) RowToColumnar +(92) Exchange Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(103) CometColumnarExchange -Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(104) ColumnarToRow [codegen id : 26] -Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] - -(105) HashAggregate [codegen id : 26] +(93) HashAggregate [codegen id : 26] Input [5]: [web_site_id#111, sum#160, sum#161, sum#162, sum#163] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] -(106) Union +(94) Union -(107) HashAggregate [codegen id : 27] +(95) HashAggregate [codegen id : 27] Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] Keys [2]: [channel#37, id#38] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -(108) RowToColumnar -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] - -(109) CometColumnarExchange -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(110) ColumnarToRow [codegen id : 28] +(96) Exchange Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(111) HashAggregate [codegen id : 28] +(97) HashAggregate [codegen id : 28] Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] Results [4]: [channel#37, sum(sales#39)#141 AS sales#164, sum(returns#40)#142 AS returns#165, sum(profit#41)#143 AS profit#166] -(112) HashAggregate [codegen id : 28] +(98) HashAggregate [codegen id : 28] Input [4]: [channel#37, sales#164, returns#165, profit#166] Keys [1]: [channel#37] Functions [3]: [partial_sum(sales#164), partial_sum(returns#165), partial_sum(profit#166)] Aggregate Attributes [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] Results [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] -(113) RowToColumnar -Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] - -(114) CometColumnarExchange +(99) Exchange Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] -Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(115) ColumnarToRow [codegen id : 29] -Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] - -(116) HashAggregate [codegen id : 29] +(100) HashAggregate [codegen id : 29] Input [7]: [channel#37, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Keys [1]: [channel#37] Functions [3]: [sum(sales#164), sum(returns#165), sum(profit#166)] Aggregate Attributes [3]: [sum(sales#164)#179, sum(returns#165)#180, sum(profit#166)#181] Results [5]: [channel#37, null AS id#182, sum(sales#164)#179 AS sum(sales)#183, sum(returns#165)#180 AS sum(returns)#184, sum(profit#166)#181 AS sum(profit)#185] -(117) ReusedExchange [Reuses operator id: 20] +(101) ReusedExchange [Reuses operator id: 19] Output [5]: [s_store_id#24, sum#186, sum#187, sum#188, sum#189] -(118) ColumnarToRow [codegen id : 33] -Input [5]: [s_store_id#24, sum#186, sum#187, sum#188, sum#189] - -(119) HashAggregate [codegen id : 33] +(102) HashAggregate [codegen id : 33] Input [5]: [s_store_id#24, sum#186, sum#187, sum#188, sum#189] Keys [1]: [s_store_id#24] 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))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] -(120) ReusedExchange [Reuses operator id: 42] +(103) ReusedExchange [Reuses operator id: 39] Output [5]: [cp_catalog_page_id#65, sum#190, sum#191, sum#192, sum#193] -(121) ColumnarToRow [codegen id : 37] -Input [5]: [cp_catalog_page_id#65, sum#190, sum#191, sum#192, sum#193] - -(122) HashAggregate [codegen id : 37] +(104) HashAggregate [codegen id : 37] Input [5]: [cp_catalog_page_id#65, sum#190, sum#191, sum#192, sum#193] Keys [1]: [cp_catalog_page_id#65] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -739,11 +659,11 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(124) CometFilter +(106) CometFilter Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Condition : isnotnull(ws_web_site_sk#83) -(125) CometProject +(107) CometProject Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] @@ -754,7 +674,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#194)] ReadSchema: struct -(127) CometBroadcastExchange +(109) CometBroadcastExchange Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] @@ -765,166 +685,144 @@ 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 -(129) CometFilter +(111) CometFilter Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) -(130) CometProject +(112) CometProject Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(131) CometBroadcastHashJoin +(113) CometBroadcastHashJoin Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner -(132) CometProject +(114) CometProject Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -(133) CometUnion +(115) CometUnion Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] -(134) ColumnarToRow [codegen id : 40] +(116) ColumnarToRow [codegen id : 40] Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -(135) ReusedExchange [Reuses operator id: 168] +(117) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#109] -(136) BroadcastHashJoin [codegen id : 40] +(118) BroadcastHashJoin [codegen id : 40] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(137) Project [codegen id : 40] +(119) Project [codegen id : 40] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] -(138) ReusedExchange [Reuses operator id: 63] +(120) ReusedExchange [Reuses operator id: 59] Output [2]: [web_site_sk#110, web_site_id#111] -(139) BroadcastHashJoin [codegen id : 40] +(121) BroadcastHashJoin [codegen id : 40] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(140) Project [codegen id : 40] +(122) Project [codegen id : 40] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] -(141) HashAggregate [codegen id : 40] +(123) HashAggregate [codegen id : 40] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] Keys [1]: [web_site_id#111] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum#195, sum#196, sum#197, sum#198] Results [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] -(142) RowToColumnar -Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] - -(143) CometColumnarExchange +(124) Exchange Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(144) ColumnarToRow [codegen id : 41] -Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] - -(145) HashAggregate [codegen id : 41] +(125) HashAggregate [codegen id : 41] Input [5]: [web_site_id#111, sum#199, sum#200, sum#201, sum#202] Keys [1]: [web_site_id#111] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] -(146) Union +(126) Union -(147) HashAggregate [codegen id : 42] +(127) HashAggregate [codegen id : 42] Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] Keys [2]: [channel#37, id#38] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -(148) RowToColumnar +(128) Exchange Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(149) CometColumnarExchange -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(150) ColumnarToRow [codegen id : 43] -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] - -(151) HashAggregate [codegen id : 43] +(129) HashAggregate [codegen id : 43] Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] Results [3]: [sum(sales#39)#141 AS sales#164, sum(returns#40)#142 AS returns#165, sum(profit#41)#143 AS profit#166] -(152) HashAggregate [codegen id : 43] +(130) HashAggregate [codegen id : 43] Input [3]: [sales#164, returns#165, profit#166] Keys: [] Functions [3]: [partial_sum(sales#164), partial_sum(returns#165), partial_sum(profit#166)] Aggregate Attributes [6]: [sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208] Results [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] -(153) RowToColumnar -Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] - -(154) CometColumnarExchange -Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(155) ColumnarToRow [codegen id : 44] +(131) Exchange Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(156) HashAggregate [codegen id : 44] +(132) HashAggregate [codegen id : 44] Input [6]: [sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214] Keys: [] Functions [3]: [sum(sales#164), sum(returns#165), sum(profit#166)] Aggregate Attributes [3]: [sum(sales#164)#215, sum(returns#165)#216, sum(profit#166)#217] Results [5]: [null AS channel#218, null AS id#219, sum(sales#164)#215 AS sum(sales)#220, sum(returns#165)#216 AS sum(returns)#221, sum(profit#166)#217 AS sum(profit)#222] -(157) Union +(133) Union -(158) HashAggregate [codegen id : 45] +(134) HashAggregate [codegen id : 45] Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] Functions: [] Aggregate Attributes: [] Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -(159) RowToColumnar +(135) Exchange Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Arguments: hashpartitioning(channel#37, id#38, sales#144, returns#145, profit#146, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(160) CometColumnarExchange -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Arguments: hashpartitioning(channel#37, id#38, sales#144, returns#145, profit#146, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(161) CometHashAggregate +(136) HashAggregate [codegen id : 46] Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -(162) CometTakeOrderedAndProject -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#37 ASC NULLS FIRST,id#38 ASC NULLS FIRST], output=[channel#37,id#38,sales#144,returns#145,profit#146]), 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#144, returns#145, profit#146] - -(163) ColumnarToRow [codegen id : 46] +(137) TakeOrderedAndProject Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#144, returns#145, profit#146] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (168) -+- * ColumnarToRow (167) - +- CometProject (166) - +- CometFilter (165) - +- CometScan parquet spark_catalog.default.date_dim (164) +BroadcastExchange (142) ++- * ColumnarToRow (141) + +- CometProject (140) + +- CometFilter (139) + +- CometScan parquet spark_catalog.default.date_dim (138) (unknown) Scan parquet spark_catalog.default.date_dim @@ -934,37 +832,37 @@ 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 -(165) CometFilter +(139) CometFilter Input [2]: [d_date_sk#22, d_date#223] Condition : (((isnotnull(d_date#223) AND (d_date#223 >= 1998-08-04)) AND (d_date#223 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(166) CometProject +(140) CometProject Input [2]: [d_date_sk#22, d_date#223] Arguments: [d_date_sk#22], [d_date_sk#22] -(167) ColumnarToRow [codegen id : 1] +(141) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(168) BroadcastExchange +(142) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 48 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 -Subquery:7 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:7 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:8 Hosting operator id = 86 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:8 Hosting operator id = 76 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 -Subquery:9 Hosting operator id = 123 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:9 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 -Subquery:10 Hosting operator id = 126 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 108 Hosting Expression = wr_returned_date_sk#98 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 ab4910c4a8..ff62cb5c24 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,244 +1,218 @@ -WholeStageCodegen (46) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - RowToColumnar - WholeStageCodegen (45) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (14) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - RowToColumnar - WholeStageCodegen (13) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (46) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Exchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (45) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (14) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel,id] #2 + WholeStageCodegen (13) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + 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] InputAdapter - Union - WholeStageCodegen (4) - 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] - ColumnarToRow + Exchange [s_store_id] #3 + WholeStageCodegen (3) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan 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] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) + 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] + InputAdapter + Exchange [cp_catalog_page_id] #6 + WholeStageCodegen (7) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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] + CometScan 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 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (12) + 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] + InputAdapter + Exchange [web_site_id] #8 + WholeStageCodegen (11) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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,ws_item_sk,ws_order_number] + CometBroadcastExchange #9 + CometScan 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_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (29) + 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] + InputAdapter + Exchange [channel] #11 + WholeStageCodegen (28) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel,id] #12 + WholeStageCodegen (27) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (18) + 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] InputAdapter - CometColumnarExchange [s_store_id] #3 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan 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] - CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (8) - 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] - ColumnarToRow + ReusedExchange [s_store_id,sum,sum,sum,sum] #3 + WholeStageCodegen (22) + 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] InputAdapter - CometColumnarExchange [cp_catalog_page_id] #6 - RowToColumnar - WholeStageCodegen (7) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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] - CometScan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (12) - 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] - ColumnarToRow + ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 + WholeStageCodegen (26) + 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] InputAdapter - CometColumnarExchange [web_site_id] #8 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + Exchange [web_site_id] #13 + WholeStageCodegen (25) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + 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] + CometScan 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,ws_item_sk,ws_order_number] + CometBroadcastExchange #14 + CometScan 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_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,ws_item_sk,ws_order_number] - CometBroadcastExchange #9 - CometScan 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_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [web_site_sk,web_site_id] #10 + WholeStageCodegen (44) + 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] + InputAdapter + Exchange #15 + WholeStageCodegen (43) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel,id] #16 + WholeStageCodegen (42) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (33) + 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] + InputAdapter + ReusedExchange [s_store_id,sum,sum,sum,sum] #3 + WholeStageCodegen (37) + 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] + InputAdapter + ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 + WholeStageCodegen (41) + 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] + InputAdapter + Exchange [web_site_id] #17 + WholeStageCodegen (40) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (29) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel] #11 - RowToColumnar - WholeStageCodegen (28) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #12 - RowToColumnar - WholeStageCodegen (27) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (18) - 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] - ColumnarToRow - InputAdapter - ReusedExchange [s_store_id,sum,sum,sum,sum] #3 - WholeStageCodegen (22) - 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] - ColumnarToRow - InputAdapter - ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 - WholeStageCodegen (26) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #13 - RowToColumnar - WholeStageCodegen (25) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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,ws_item_sk,ws_order_number] - CometBroadcastExchange #14 - CometScan 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_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [web_site_sk,web_site_id] #10 - WholeStageCodegen (44) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange #15 - RowToColumnar - WholeStageCodegen (43) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #16 - RowToColumnar - WholeStageCodegen (42) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (33) - 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] - ColumnarToRow - InputAdapter - ReusedExchange [s_store_id,sum,sum,sum,sum] #3 - WholeStageCodegen (37) - 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] - ColumnarToRow - InputAdapter - ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #6 - WholeStageCodegen (41) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #17 - RowToColumnar - WholeStageCodegen (40) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometUnion - 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] - CometScan 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,ws_item_sk,ws_order_number] - CometBroadcastExchange #18 - CometScan 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_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [web_site_sk,web_site_id] #10 + CometUnion + 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] + CometScan 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,ws_item_sk,ws_order_number] + CometBroadcastExchange #18 + CometScan 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_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [web_site_sk,web_site_id] #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index 7c9d8e2e4e..4146a59977 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -1,45 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Filter (40) - +- * HashAggregate (39) - +- * ColumnarToRow (38) - +- CometColumnarExchange (37) - +- RowToColumnar (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer_address (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometFilter (27) - +- CometHashAggregate (26) - +- CometColumnarExchange (25) - +- CometHashAggregate (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.item (22) +TakeOrderedAndProject (39) ++- * Filter (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer_address (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store_sales (10) + : +- ReusedExchange (16) + +- BroadcastExchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * ColumnarToRow (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- BroadcastExchange (29) + +- * Filter (28) + +- * HashAggregate (27) + +- Exchange (26) + +- * ColumnarToRow (25) + +- CometHashAggregate (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.item (22) (unknown) Scan parquet spark_catalog.default.customer_address @@ -53,7 +51,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#1, ca_state#2] Condition : isnotnull(ca_address_sk#1) -(3) ColumnarToRow [codegen id : 6] +(3) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#1, ca_state#2] (unknown) Scan parquet spark_catalog.default.customer @@ -74,13 +72,13 @@ Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 6] +(8) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ca_address_sk#1] Right keys [1]: [c_current_addr_sk#4] Join type: Inner Join condition: None -(9) Project [codegen id : 6] +(9) Project [codegen id : 7] Output [2]: [ca_state#2, c_customer_sk#3] Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4] @@ -103,26 +101,26 @@ Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 6] +(14) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: Inner Join condition: None -(15) Project [codegen id : 6] +(15) Project [codegen id : 7] Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -(16) ReusedExchange [Reuses operator id: 46] +(16) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#9] -(17) BroadcastHashJoin [codegen id : 6] +(17) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(18) Project [codegen id : 6] +(18) Project [codegen id : 7] Output [2]: [ca_state#2, ss_item_sk#5] Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] @@ -137,7 +135,7 @@ ReadSchema: struct Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10)) -(21) ColumnarToRow [codegen id : 5] +(21) ColumnarToRow [codegen id : 6] Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] (unknown) Scan parquet spark_catalog.default.item @@ -156,154 +154,152 @@ Input [2]: [i_current_price#13, i_category#14] Keys [1]: [i_category#14] Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] -(25) CometColumnarExchange +(25) ColumnarToRow [codegen id : 4] Input [3]: [i_category#14, sum#15, count#16] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(26) CometHashAggregate +(26) Exchange +Input [3]: [i_category#14, sum#15, count#16] +Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 5] Input [3]: [i_category#14, sum#15, count#16] Keys [1]: [i_category#14] Functions [1]: [avg(UnscaledValue(i_current_price#13))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#17] +Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#17 / 100.0) as decimal(11,6)) AS avg(i_current_price)#18, i_category#14] -(27) CometFilter -Input [2]: [avg(i_current_price)#17, i_category#14] -Condition : isnotnull(avg(i_current_price)#17) - -(28) ColumnarToRow [codegen id : 4] -Input [2]: [avg(i_current_price)#17, i_category#14] +(28) Filter [codegen id : 5] +Input [2]: [avg(i_current_price)#18, i_category#14] +Condition : isnotnull(avg(i_current_price)#18) (29) BroadcastExchange -Input [2]: [avg(i_current_price)#17, i_category#14] +Input [2]: [avg(i_current_price)#18, i_category#14] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] -(30) BroadcastHashJoin [codegen id : 5] +(30) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#12] Right keys [1]: [i_category#14] Join type: Inner -Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#17)) +Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#18)) -(31) Project [codegen id : 5] +(31) Project [codegen id : 6] Output [1]: [i_item_sk#10] -Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#17, i_category#14] +Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#18, i_category#14] (32) BroadcastExchange Input [1]: [i_item_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 6] +(33) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#5] Right keys [1]: [i_item_sk#10] Join type: Inner Join condition: None -(34) Project [codegen id : 6] +(34) Project [codegen id : 7] Output [1]: [ca_state#2] Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#10] -(35) HashAggregate [codegen id : 6] +(35) HashAggregate [codegen id : 7] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_state#2, count#19] - -(36) RowToColumnar -Input [2]: [ca_state#2, count#19] +Aggregate Attributes [1]: [count#19] +Results [2]: [ca_state#2, count#20] -(37) CometColumnarExchange -Input [2]: [ca_state#2, count#19] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(36) Exchange +Input [2]: [ca_state#2, count#20] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(38) ColumnarToRow [codegen id : 7] -Input [2]: [ca_state#2, count#19] - -(39) HashAggregate [codegen id : 7] -Input [2]: [ca_state#2, count#19] +(37) HashAggregate [codegen id : 8] +Input [2]: [ca_state#2, count#20] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#20] -Results [3]: [ca_state#2 AS state#21, count(1)#20 AS cnt#22, ca_state#2] +Aggregate Attributes [1]: [count(1)#21] +Results [3]: [ca_state#2 AS state#22, count(1)#21 AS cnt#23, ca_state#2] -(40) Filter [codegen id : 7] -Input [3]: [state#21, cnt#22, ca_state#2] -Condition : (cnt#22 >= 10) +(38) Filter [codegen id : 8] +Input [3]: [state#22, cnt#23, ca_state#2] +Condition : (cnt#23 >= 10) -(41) TakeOrderedAndProject -Input [3]: [state#21, cnt#22, ca_state#2] -Arguments: 100, [cnt#22 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#21, cnt#22] +(39) TakeOrderedAndProject +Input [3]: [state#22, cnt#23, ca_state#2] +Arguments: 100, [cnt#23 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#22, cnt#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (46) -+- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.date_dim (42) +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#23] +Output [2]: [d_date_sk#9, d_month_seq#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#23] -Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-subquery#24, [id=#25])) AND isnotnull(d_date_sk#9)) +(41) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#24] +Condition : ((isnotnull(d_month_seq#24) AND (d_month_seq#24 = Subquery scalar-subquery#25, [id=#26])) AND isnotnull(d_date_sk#9)) -(44) CometProject -Input [2]: [d_date_sk#9, d_month_seq#23] +(42) CometProject +Input [2]: [d_date_sk#9, d_month_seq#24] Arguments: [d_date_sk#9], [d_date_sk#9] -(45) ColumnarToRow [codegen id : 1] +(43) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(46) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#24, [id=#25] -* ColumnarToRow (53) -+- CometHashAggregate (52) - +- CometColumnarExchange (51) - +- CometHashAggregate (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan parquet spark_catalog.default.date_dim (47) +Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#25, [id=#26] +* HashAggregate (51) ++- Exchange (50) + +- * ColumnarToRow (49) + +- CometHashAggregate (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] +Output [3]: [d_month_seq#27, d_year#28, d_moy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(48) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) +(46) CometFilter +Input [3]: [d_month_seq#27, d_year#28, d_moy#29] +Condition : (((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2000)) AND (d_moy#29 = 1)) -(49) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] +(47) CometProject +Input [3]: [d_month_seq#27, d_year#28, d_moy#29] +Arguments: [d_month_seq#27], [d_month_seq#27] -(50) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +(48) CometHashAggregate +Input [1]: [d_month_seq#27] +Keys [1]: [d_month_seq#27] Functions: [] -(51) CometColumnarExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +(49) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#27] -(52) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] +(50) Exchange +Input [1]: [d_month_seq#27] +Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(53) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] +(51) HashAggregate [codegen id : 2] +Input [1]: [d_month_seq#27] +Keys [1]: [d_month_seq#27] +Functions: [] +Aggregate Attributes: [] +Results [1]: [d_month_seq#27] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index b30a459724..57ef6ebd5f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -1,76 +1,78 @@ TakeOrderedAndProject [cnt,ca_state,state] - WholeStageCodegen (7) + WholeStageCodegen (8) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] - ColumnarToRow - InputAdapter - CometColumnarExchange [ca_state] #1 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + Exchange [ca_state] #1 + WholeStageCodegen (7) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ca_state,ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ca_state,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [ca_state,c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) + BroadcastExchange #2 + WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometColumnarExchange [d_month_seq] #5 + CometFilter [c_current_addr_sk,c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (2) + HashAggregate [d_month_seq] + InputAdapter + Exchange [d_month_seq] #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] CometFilter [d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow - InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [avg(i_current_price)] - CometHashAggregate [i_category,sum,count] - CometColumnarExchange [i_category] #8 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Project [i_item_sk] + BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] + ColumnarToRow + InputAdapter + CometFilter [i_current_price,i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + Filter [avg(i_current_price)] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + InputAdapter + Exchange [i_category] #8 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter CometHashAggregate [i_category,i_current_price] CometFilter [i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index fdfaa1e2cf..b4d1007622 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -1,189 +1,185 @@ == Physical Plan == -* ColumnarToRow (185) -+- CometSort (184) - +- CometColumnarExchange (183) - +- RowToColumnar (182) - +- * Project (181) - +- * SortMergeJoin Inner (180) - :- * ColumnarToRow (110) - : +- CometSort (109) - : +- CometColumnarExchange (108) - : +- RowToColumnar (107) - : +- * HashAggregate (106) - : +- * HashAggregate (105) - : +- * Project (104) - : +- * BroadcastHashJoin Inner BuildRight (103) - : :- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * Project (94) - : : : +- * BroadcastHashJoin Inner BuildRight (93) - : : : :- * Project (88) - : : : : +- * BroadcastHashJoin Inner BuildRight (87) - : : : : :- * Project (85) - : : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : : :- * Project (79) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : :- * Project (76) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : : : : : :- * Project (70) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (69) - : : : : : : : : :- * Project (64) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : : : : : : :- * Project (61) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : : : : : : : :- * Project (55) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : : : : : : : :- * Project (52) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : : : : : : : : : : :- * Project (46) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : : : : : : : : : : :- * Project (40) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : : : : : : : : : : : : :- * Project (34) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : : : : : : : : : : : :- * ColumnarToRow (31) - : : : : : : : : : : : : : : : : +- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- ReusedExchange (32) - : : : : : : : : : : : : : : +- BroadcastExchange (38) - : : : : : : : : : : : : : : +- * ColumnarToRow (37) - : : : : : : : : : : : : : : +- CometFilter (36) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (35) - : : : : : : : : : : : : : +- BroadcastExchange (44) - : : : : : : : : : : : : : +- * ColumnarToRow (43) - : : : : : : : : : : : : : +- CometFilter (42) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) - : : : : : : : : : : : : +- BroadcastExchange (50) - : : : : : : : : : : : : +- * ColumnarToRow (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (53) - : : : : : : : : : : +- BroadcastExchange (59) - : : : : : : : : : : +- * ColumnarToRow (58) - : : : : : : : : : : +- CometFilter (57) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (56) - : : : : : : : : : +- ReusedExchange (62) - : : : : : : : : +- BroadcastExchange (68) - : : : : : : : : +- * ColumnarToRow (67) - : : : : : : : : +- CometFilter (66) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (65) - : : : : : : : +- BroadcastExchange (74) - : : : : : : : +- * ColumnarToRow (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (77) - : : : : : +- BroadcastExchange (83) - : : : : : +- * ColumnarToRow (82) - : : : : : +- CometFilter (81) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (80) - : : : : +- ReusedExchange (86) - : : : +- BroadcastExchange (92) - : : : +- * ColumnarToRow (91) - : : : +- CometFilter (90) - : : : +- CometScan parquet spark_catalog.default.income_band (89) - : : +- ReusedExchange (95) - : +- BroadcastExchange (102) - : +- * ColumnarToRow (101) - : +- CometProject (100) - : +- CometFilter (99) - : +- CometScan parquet spark_catalog.default.item (98) - +- * ColumnarToRow (179) - +- CometSort (178) - +- CometColumnarExchange (177) - +- RowToColumnar (176) - +- * HashAggregate (175) - +- * HashAggregate (174) - +- * Project (173) - +- * BroadcastHashJoin Inner BuildRight (172) - :- * Project (170) - : +- * BroadcastHashJoin Inner BuildRight (169) - : :- * Project (167) - : : +- * BroadcastHashJoin Inner BuildRight (166) - : : :- * Project (164) - : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : :- * Project (161) - : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : :- * Project (158) - : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : :- * Project (155) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : :- * Project (152) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : :- * Project (149) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) - : : : : : : : : :- * Project (146) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) - : : : : : : : : : :- * Project (143) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : : : : : : : : :- * Project (140) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : : : : : : : : :- * Project (137) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) - : : : : : : : : : : : : : : :- * ColumnarToRow (128) - : : : : : : : : : : : : : : : +- CometProject (127) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (126) - : : : : : : : : : : : : : : : :- CometSort (120) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (119) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) - : : : : : : : : : : : : : : : : : +- CometFilter (112) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometFilter (115) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) - : : : : : : : : : : : : : : : +- CometSort (125) - : : : : : : : : : : : : : : : +- CometProject (124) - : : : : : : : : : : : : : : : +- CometFilter (123) - : : : : : : : : : : : : : : : +- CometHashAggregate (122) - : : : : : : : : : : : : : : : +- ReusedExchange (121) - : : : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : +- ReusedExchange (147) - : : : : : : : +- ReusedExchange (150) - : : : : : : +- ReusedExchange (153) - : : : : : +- ReusedExchange (156) - : : : : +- ReusedExchange (159) - : : : +- ReusedExchange (162) - : : +- ReusedExchange (165) - : +- ReusedExchange (168) - +- ReusedExchange (171) +* Sort (181) ++- Exchange (180) + +- * Project (179) + +- * SortMergeJoin Inner (178) + :- * Sort (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- * Project (106) + : +- * BroadcastHashJoin Inner BuildRight (105) + : :- * Project (99) + : : +- * BroadcastHashJoin Inner BuildRight (98) + : : :- * Project (96) + : : : +- * BroadcastHashJoin Inner BuildRight (95) + : : : :- * Project (90) + : : : : +- * BroadcastHashJoin Inner BuildRight (89) + : : : : :- * Project (87) + : : : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : : : :- * Project (81) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) + : : : : : : :- * Project (78) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) + : : : : : : : :- * Project (72) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) + : : : : : : : : :- * Project (66) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) + : : : : : : : : : :- * Project (63) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : : : : : : : : :- * Project (57) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : : : : : : : : : :- * Project (54) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) + : : : : : : : : : : : : :- * Project (48) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : : : : : : : : : : :- * Project (42) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : : : : : : : : : : : :- * Project (36) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : : : : : : : : : : : :- * Project (33) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) + : : : : : : : : : : : : : : : : :- * Sort (11) + : : : : : : : : : : : : : : : : : +- Exchange (10) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- * Sort (31) + : : : : : : : : : : : : : : : : +- * Project (30) + : : : : : : : : : : : : : : : : +- * Filter (29) + : : : : : : : : : : : : : : : : +- * HashAggregate (28) + : : : : : : : : : : : : : : : : +- Exchange (27) + : : : : : : : : : : : : : : : : +- * HashAggregate (26) + : : : : : : : : : : : : : : : : +- * Project (25) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) + : : : : : : : : : : : : : : : : :- * Sort (17) + : : : : : : : : : : : : : : : : : +- Exchange (16) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) + : : : : : : : : : : : : : : : : : +- CometProject (14) + : : : : : : : : : : : : : : : : : +- CometFilter (13) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : : : : : : : : : : : : : : : +- * Sort (23) + : : : : : : : : : : : : : : : : +- Exchange (22) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) + : : : : : : : : : : : : : : : : +- CometProject (20) + : : : : : : : : : : : : : : : : +- CometFilter (19) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) + : : : : : : : : : : : : : : : +- ReusedExchange (34) + : : : : : : : : : : : : : : +- BroadcastExchange (40) + : : : : : : : : : : : : : : +- * ColumnarToRow (39) + : : : : : : : : : : : : : : +- CometFilter (38) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) + : : : : : : : : : : : : : +- BroadcastExchange (46) + : : : : : : : : : : : : : +- * ColumnarToRow (45) + : : : : : : : : : : : : : +- CometFilter (44) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) + : : : : : : : : : : : : +- BroadcastExchange (52) + : : : : : : : : : : : : +- * ColumnarToRow (51) + : : : : : : : : : : : : +- CometFilter (50) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) + : : : : : : : : : : : +- ReusedExchange (55) + : : : : : : : : : : +- BroadcastExchange (61) + : : : : : : : : : : +- * ColumnarToRow (60) + : : : : : : : : : : +- CometFilter (59) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) + : : : : : : : : : +- ReusedExchange (64) + : : : : : : : : +- BroadcastExchange (70) + : : : : : : : : +- * ColumnarToRow (69) + : : : : : : : : +- CometFilter (68) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) + : : : : : : : +- BroadcastExchange (76) + : : : : : : : +- * ColumnarToRow (75) + : : : : : : : +- CometFilter (74) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) + : : : : : : +- ReusedExchange (79) + : : : : : +- BroadcastExchange (85) + : : : : : +- * ColumnarToRow (84) + : : : : : +- CometFilter (83) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) + : : : : +- ReusedExchange (88) + : : : +- BroadcastExchange (94) + : : : +- * ColumnarToRow (93) + : : : +- CometFilter (92) + : : : +- CometScan parquet spark_catalog.default.income_band (91) + : : +- ReusedExchange (97) + : +- BroadcastExchange (104) + : +- * ColumnarToRow (103) + : +- CometProject (102) + : +- CometFilter (101) + : +- CometScan parquet spark_catalog.default.item (100) + +- * Sort (177) + +- Exchange (176) + +- * HashAggregate (175) + +- * HashAggregate (174) + +- * Project (173) + +- * BroadcastHashJoin Inner BuildRight (172) + :- * Project (170) + : +- * BroadcastHashJoin Inner BuildRight (169) + : :- * Project (167) + : : +- * BroadcastHashJoin Inner BuildRight (166) + : : :- * Project (164) + : : : +- * BroadcastHashJoin Inner BuildRight (163) + : : : :- * Project (161) + : : : : +- * BroadcastHashJoin Inner BuildRight (160) + : : : : :- * Project (158) + : : : : : +- * BroadcastHashJoin Inner BuildRight (157) + : : : : : :- * Project (155) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) + : : : : : : :- * Project (152) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) + : : : : : : : :- * Project (149) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) + : : : : : : : : :- * Project (146) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) + : : : : : : : : : :- * Project (143) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) + : : : : : : : : : : :- * Project (140) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) + : : : : : : : : : : : :- * Project (137) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) + : : : : : : : : : : : : :- * Project (134) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) + : : : : : : : : : : : : : :- * Project (131) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) + : : : : : : : : : : : : : : :- * Project (128) + : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) + : : : : : : : : : : : : : : : :- * Sort (121) + : : : : : : : : : : : : : : : : +- Exchange (120) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) + : : : : : : : : : : : : : : : : +- CometProject (118) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) + : : : : : : : : : : : : : : : : : +- CometFilter (112) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) + : : : : : : : : : : : : : : : : +- CometProject (116) + : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) + : : : : : : : : : : : : : : : +- * Sort (126) + : : : : : : : : : : : : : : : +- * Project (125) + : : : : : : : : : : : : : : : +- * Filter (124) + : : : : : : : : : : : : : : : +- * HashAggregate (123) + : : : : : : : : : : : : : : : +- ReusedExchange (122) + : : : : : : : : : : : : : : +- ReusedExchange (129) + : : : : : : : : : : : : : +- ReusedExchange (132) + : : : : : : : : : : : : +- ReusedExchange (135) + : : : : : : : : : : : +- ReusedExchange (138) + : : : : : : : : : : +- ReusedExchange (141) + : : : : : : : : : +- ReusedExchange (144) + : : : : : : : : +- ReusedExchange (147) + : : : : : : : +- ReusedExchange (150) + : : : : : : +- ReusedExchange (153) + : : : : : +- ReusedExchange (156) + : : : : +- ReusedExchange (159) + : : : +- ReusedExchange (162) + : : +- ReusedExchange (165) + : +- ReusedExchange (168) + +- ReusedExchange (171) (unknown) Scan parquet spark_catalog.default.store_sales @@ -226,13 +222,16 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) CometColumnarExchange +(9) ColumnarToRow [codegen id : 1] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) CometSort +(10) Exchange Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(11) Sort [codegen id : 2] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] @@ -241,21 +240,24 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(12) CometFilter +(13) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(13) CometProject +(14) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(14) CometColumnarExchange +(15) ColumnarToRow [codegen id : 3] +Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] + +(16) Exchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometSort +(17) Sort [codegen id : 4] Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] +Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] @@ -264,808 +266,799 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(17) CometFilter +(19) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(18) CometProject +(20) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(19) CometColumnarExchange +(21) ColumnarToRow [codegen id : 5] Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) CometSort +(22) Exchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner +(23) Sort [codegen id : 6] +Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 -(22) CometProject +(24) SortMergeJoin [codegen id : 7] +Left keys [2]: [cs_item_sk#17, cs_order_number#18] +Right keys [2]: [cr_item_sk#21, cr_order_number#22] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 7] +Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(23) CometHashAggregate +(26) HashAggregate [codegen id : 7] Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] +Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(24) CometColumnarExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(27) Exchange +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] +(28) HashAggregate [codegen id : 8] +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] +Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) +(29) Filter [codegen id : 8] +Input [3]: [cs_item_sk#17, sale#35, refund#36] +Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] +(30) Project [codegen id : 8] +Output [1]: [cs_item_sk#17] +Input [3]: [cs_item_sk#17, sale#35, refund#36] -(28) CometSort +(31) Sort [codegen id : 8] Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] +Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner +(32) SortMergeJoin [codegen id : 24] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#17] +Join type: Inner +Join condition: None -(30) CometProject +(33) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) ColumnarToRow [codegen id : 16] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(32) ReusedExchange [Reuses operator id: 189] -Output [2]: [d_date_sk#32, d_year#33] +(34) ReusedExchange [Reuses operator id: 185] +Output [2]: [d_date_sk#37, d_year#38] -(33) BroadcastHashJoin [codegen id : 16] +(35) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#32] +Right keys [1]: [d_date_sk#37] Join type: Inner Join condition: None -(34) Project [codegen id : 16] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] +(36) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] (unknown) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(36) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(s_zip#36)) +(38) CometFilter +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) -(37) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +(39) ColumnarToRow [codegen id : 10] +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -(38) BroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +(40) BroadcastExchange +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 16] +(41) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#34] +Right keys [1]: [s_store_sk#39] Join type: Inner Join condition: None -(40) Project [codegen id : 16] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#36] +(42) Project [codegen id : 24] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] (unknown) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(42) CometFilter -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -Condition : (((((isnotnull(c_customer_sk#37) AND isnotnull(c_first_sales_date_sk#42)) AND isnotnull(c_first_shipto_date_sk#41)) AND isnotnull(c_current_cdemo_sk#38)) AND isnotnull(c_current_hdemo_sk#39)) AND isnotnull(c_current_addr_sk#40)) +(44) CometFilter +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) -(43) ColumnarToRow [codegen id : 3] -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(45) ColumnarToRow [codegen id : 11] +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -(44) BroadcastExchange -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(46) BroadcastExchange +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 16] +(47) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#37] +Right keys [1]: [c_customer_sk#42] Join type: Inner Join condition: None -(46) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(48) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#48, d_year#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : isnotnull(d_date_sk#43) +(50) CometFilter +Input [2]: [d_date_sk#48, d_year#49] +Condition : isnotnull(d_date_sk#48) -(49) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#43, d_year#44] +(51) ColumnarToRow [codegen id : 12] +Input [2]: [d_date_sk#48, d_year#49] -(50) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] +(52) BroadcastExchange +Input [2]: [d_date_sk#48, d_year#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_first_sales_date_sk#42] -Right keys [1]: [d_date_sk#43] +(53) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_sales_date_sk#47] +Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42, d_date_sk#43, d_year#44] +(54) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] -(53) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#45, d_year#46] +(55) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#50, d_year#51] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_first_shipto_date_sk#41] -Right keys [1]: [d_date_sk#45] +(56) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_shipto_date_sk#46] +Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(55) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44, d_date_sk#45, d_year#46] +(57) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] (unknown) Scan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#47, cd_marital_status#48] +Output [2]: [cd_demo_sk#52, cd_marital_status#53] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(57) CometFilter -Input [2]: [cd_demo_sk#47, cd_marital_status#48] -Condition : (isnotnull(cd_demo_sk#47) AND isnotnull(cd_marital_status#48)) +(59) CometFilter +Input [2]: [cd_demo_sk#52, cd_marital_status#53] +Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) -(58) ColumnarToRow [codegen id : 6] -Input [2]: [cd_demo_sk#47, cd_marital_status#48] +(60) ColumnarToRow [codegen id : 14] +Input [2]: [cd_demo_sk#52, cd_marital_status#53] -(59) BroadcastExchange -Input [2]: [cd_demo_sk#47, cd_marital_status#48] +(61) BroadcastExchange +Input [2]: [cd_demo_sk#52, cd_marital_status#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 16] +(62) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#47] +Right keys [1]: [cd_demo_sk#52] Join type: Inner Join condition: None -(61) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_demo_sk#47, cd_marital_status#48] +(63) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] -(62) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#49, cd_marital_status#50] +(64) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#54, cd_marital_status#55] -(63) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_cdemo_sk#38] -Right keys [1]: [cd_demo_sk#49] +(65) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_cdemo_sk#43] +Right keys [1]: [cd_demo_sk#54] Join type: Inner -Join condition: NOT (cd_marital_status#48 = cd_marital_status#50) +Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) -(64) Project [codegen id : 16] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48, cd_demo_sk#49, cd_marital_status#50] +(66) Project [codegen id : 24] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] (unknown) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#51] +Output [1]: [p_promo_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(66) CometFilter -Input [1]: [p_promo_sk#51] -Condition : isnotnull(p_promo_sk#51) +(68) CometFilter +Input [1]: [p_promo_sk#56] +Condition : isnotnull(p_promo_sk#56) -(67) ColumnarToRow [codegen id : 8] -Input [1]: [p_promo_sk#51] +(69) ColumnarToRow [codegen id : 16] +Input [1]: [p_promo_sk#56] -(68) BroadcastExchange -Input [1]: [p_promo_sk#51] +(70) BroadcastExchange +Input [1]: [p_promo_sk#56] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(69) BroadcastHashJoin [codegen id : 16] +(71) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#51] +Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(70) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, p_promo_sk#51] +(72) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] (unknown) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(72) CometFilter -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] -Condition : (isnotnull(hd_demo_sk#52) AND isnotnull(hd_income_band_sk#53)) +(74) CometFilter +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) -(73) ColumnarToRow [codegen id : 9] -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +(75) ColumnarToRow [codegen id : 17] +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -(74) BroadcastExchange -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +(76) BroadcastExchange +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(75) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#52] +Right keys [1]: [hd_demo_sk#57] Join type: Inner Join condition: None -(76) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_demo_sk#52, hd_income_band_sk#53] +(78) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] -(77) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] +(79) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] -(78) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_hdemo_sk#39] -Right keys [1]: [hd_demo_sk#54] +(80) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_hdemo_sk#44] +Right keys [1]: [hd_demo_sk#59] Join type: Inner Join condition: None -(79) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_demo_sk#54, hd_income_band_sk#55] +(81) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] (unknown) Scan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(81) CometFilter -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -Condition : isnotnull(ca_address_sk#56) +(83) CometFilter +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Condition : isnotnull(ca_address_sk#61) -(82) ColumnarToRow [codegen id : 11] -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(84) ColumnarToRow [codegen id : 19] +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(83) BroadcastExchange -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(85) BroadcastExchange +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] -(84) BroadcastHashJoin [codegen id : 16] +(86) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#56] +Right keys [1]: [ca_address_sk#61] Join type: Inner Join condition: None -(85) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(87) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(86) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(88) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -(87) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#61] +(89) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_addr_sk#45] +Right keys [1]: [ca_address_sk#66] Join type: Inner Join condition: None -(88) Project [codegen id : 16] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(90) Project [codegen id : 24] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] (unknown) Scan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#66] +Output [1]: [ib_income_band_sk#71] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(90) CometFilter -Input [1]: [ib_income_band_sk#66] -Condition : isnotnull(ib_income_band_sk#66) +(92) CometFilter +Input [1]: [ib_income_band_sk#71] +Condition : isnotnull(ib_income_band_sk#71) -(91) ColumnarToRow [codegen id : 13] -Input [1]: [ib_income_band_sk#66] +(93) ColumnarToRow [codegen id : 21] +Input [1]: [ib_income_band_sk#71] -(92) BroadcastExchange -Input [1]: [ib_income_band_sk#66] +(94) BroadcastExchange +Input [1]: [ib_income_band_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(93) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [hd_income_band_sk#53] -Right keys [1]: [ib_income_band_sk#66] +(95) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#58] +Right keys [1]: [ib_income_band_sk#71] Join type: Inner Join condition: None -(94) Project [codegen id : 16] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#66] +(96) Project [codegen id : 24] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] -(95) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#67] +(97) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#72] -(96) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [hd_income_band_sk#55] -Right keys [1]: [ib_income_band_sk#67] +(98) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#60] +Right keys [1]: [ib_income_band_sk#72] Join type: Inner Join condition: None -(97) Project [codegen id : 16] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#67] +(99) Project [codegen id : 24] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] (unknown) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(99) CometFilter -Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] -Condition : ((((((isnotnull(i_current_price#69) AND i_color#70 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#69 >= 64.00)) AND (i_current_price#69 <= 74.00)) AND (i_current_price#69 >= 65.00)) AND (i_current_price#69 <= 79.00)) AND isnotnull(i_item_sk#68)) +(101) CometFilter +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) -(100) CometProject -Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] -Arguments: [i_item_sk#68, i_product_name#71], [i_item_sk#68, i_product_name#71] +(102) CometProject +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] -(101) ColumnarToRow [codegen id : 15] -Input [2]: [i_item_sk#68, i_product_name#71] +(103) ColumnarToRow [codegen id : 23] +Input [2]: [i_item_sk#73, i_product_name#76] -(102) BroadcastExchange -Input [2]: [i_item_sk#68, i_product_name#71] +(104) BroadcastExchange +Input [2]: [i_item_sk#73, i_product_name#76] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -(103) BroadcastHashJoin [codegen id : 16] +(105) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#68] +Right keys [1]: [i_item_sk#73] Join type: Inner Join condition: None -(104) Project [codegen id : 16] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +(106) Project [codegen id : 24] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -(105) HashAggregate [codegen id : 16] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] +(107) HashAggregate [codegen id : 24] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#72, sum#73, sum#74, sum#75] -Results [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] +Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] +Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -(106) HashAggregate [codegen id : 16] -Input [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] -Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] +(108) HashAggregate [codegen id : 24] +Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#9))#81, sum(UnscaledValue(ss_list_price#10))#82, sum(UnscaledValue(ss_coupon_amt#11))#83] -Results [17]: [i_product_name#71 AS product_name#84, i_item_sk#68 AS item_sk#85, s_store_name#35 AS store_name#86, s_zip#36 AS store_zip#87, ca_street_number#57 AS b_street_number#88, ca_street_name#58 AS b_streen_name#89, ca_city#59 AS b_city#90, ca_zip#60 AS b_zip#91, ca_street_number#62 AS c_street_number#92, ca_street_name#63 AS c_street_name#93, ca_city#64 AS c_city#94, ca_zip#65 AS c_zip#95, d_year#33 AS syear#96, count(1)#80 AS cnt#97, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#81,17,2) AS s1#98, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#82,17,2) AS s2#99, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#83,17,2) AS s3#100] - -(107) RowToColumnar -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] +Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(108) CometColumnarExchange -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(109) Exchange +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(109) CometSort -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100], [item_sk#85 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, store_zip#87 ASC NULLS FIRST] - -(110) ColumnarToRow [codegen id : 17] -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] +(110) Sort [codegen id : 25] +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#112), dynamicpruningexpression(ss_sold_date_sk#112 IN dynamicpruning#113)] +PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (112) CometFilter -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Condition : (((((((isnotnull(ss_item_sk#101) AND isnotnull(ss_ticket_number#108)) AND isnotnull(ss_store_sk#106)) AND isnotnull(ss_customer_sk#102)) AND isnotnull(ss_cdemo_sk#103)) AND isnotnull(ss_promo_sk#107)) AND isnotnull(ss_hdemo_sk#104)) AND isnotnull(ss_addr_sk#105)) +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) (113) CometBroadcastExchange -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] (unknown) Scan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] +Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (115) CometFilter -Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] -Condition : (isnotnull(sr_item_sk#114) AND isnotnull(sr_ticket_number#115)) +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) (116) CometProject -Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] -Arguments: [sr_item_sk#114, sr_ticket_number#115], [sr_item_sk#114, sr_ticket_number#115] +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] (117) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Right output [2]: [sr_item_sk#114, sr_ticket_number#115] -Arguments: [ss_item_sk#101, ss_ticket_number#108], [sr_item_sk#114, sr_ticket_number#115], Inner +Left output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Right output [2]: [sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner (118) CometProject -Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, sr_item_sk#114, sr_ticket_number#115] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -(119) CometColumnarExchange -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(119) ColumnarToRow [codegen id : 26] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -(120) CometSort -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101 ASC NULLS FIRST] +(120) Exchange +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(121) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] +(121) Sort [codegen id : 27] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 -(122) CometHashAggregate -Input [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] -Keys [1]: [cs_item_sk#117] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#121)), sum(((cr_refunded_cash#122 + cr_reversed_charge#123) + cr_store_credit#124))] +(122) ReusedExchange [Reuses operator id: 27] +Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -(123) CometFilter -Input [3]: [cs_item_sk#117, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) +(123) HashAggregate [codegen id : 33] +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +Keys [1]: [cs_item_sk#122] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] +Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#35, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#36] -(124) CometProject -Input [3]: [cs_item_sk#117, sale#30, refund#31] -Arguments: [cs_item_sk#117], [cs_item_sk#117] +(124) Filter [codegen id : 33] +Input [3]: [cs_item_sk#122, sale#35, refund#36] +Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) -(125) CometSort -Input [1]: [cs_item_sk#117] -Arguments: [cs_item_sk#117], [cs_item_sk#117 ASC NULLS FIRST] +(125) Project [codegen id : 33] +Output [1]: [cs_item_sk#122] +Input [3]: [cs_item_sk#122, sale#35, refund#36] -(126) CometSortMergeJoin -Left output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Right output [1]: [cs_item_sk#117] -Arguments: [ss_item_sk#101], [cs_item_sk#117], Inner +(126) Sort [codegen id : 33] +Input [1]: [cs_item_sk#122] +Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 -(127) CometProject -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, cs_item_sk#117] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(127) SortMergeJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [cs_item_sk#122] +Join type: Inner +Join condition: None -(128) ColumnarToRow [codegen id : 33] -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(128) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] -(129) ReusedExchange [Reuses operator id: 193] -Output [2]: [d_date_sk#125, d_year#126] +(129) ReusedExchange [Reuses operator id: 189] +Output [2]: [d_date_sk#130, d_year#131] -(130) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_date_sk#112] -Right keys [1]: [d_date_sk#125] +(130) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_sold_date_sk#117] +Right keys [1]: [d_date_sk#130] Join type: Inner Join condition: None -(131) Project [codegen id : 33] -Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126] -Input [13]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, d_date_sk#125, d_year#126] +(131) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] +Input [13]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] -(132) ReusedExchange [Reuses operator id: 38] -Output [3]: [s_store_sk#127, s_store_name#128, s_zip#129] +(132) ReusedExchange [Reuses operator id: 40] +Output [3]: [s_store_sk#132, s_store_name#133, s_zip#134] -(133) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#106] -Right keys [1]: [s_store_sk#127] +(133) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_store_sk#111] +Right keys [1]: [s_store_sk#132] Join type: Inner Join condition: None -(134) Project [codegen id : 33] -Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129] -Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_sk#127, s_store_name#128, s_zip#129] +(134) Project [codegen id : 49] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#134] -(135) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] +(135) ReusedExchange [Reuses operator id: 46] +Output [6]: [c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -(136) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_customer_sk#102] -Right keys [1]: [c_customer_sk#130] +(136) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_customer_sk#107] +Right keys [1]: [c_customer_sk#135] Join type: Inner Join condition: None -(137) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -Input [18]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] +(137) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -(138) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#136, d_year#137] +(138) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#141, d_year#142] -(139) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_first_sales_date_sk#135] -Right keys [1]: [d_date_sk#136] +(139) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_sales_date_sk#140] +Right keys [1]: [d_date_sk#141] Join type: Inner Join condition: None -(140) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135, d_date_sk#136, d_year#137] +(140) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140, d_date_sk#141, d_year#142] -(141) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#138, d_year#139] +(141) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#143, d_year#144] -(142) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_first_shipto_date_sk#134] -Right keys [1]: [d_date_sk#138] +(142) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_shipto_date_sk#139] +Right keys [1]: [d_date_sk#143] Join type: Inner Join condition: None -(143) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137, d_date_sk#138, d_year#139] +(143) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142, d_date_sk#143, d_year#144] -(144) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#140, cd_marital_status#141] +(144) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#145, cd_marital_status#146] -(145) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_cdemo_sk#103] -Right keys [1]: [cd_demo_sk#140] +(145) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_cdemo_sk#108] +Right keys [1]: [cd_demo_sk#145] Join type: Inner Join condition: None -(146) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_demo_sk#140, cd_marital_status#141] +(146) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_demo_sk#145, cd_marital_status#146] -(147) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#142, cd_marital_status#143] +(147) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#147, cd_marital_status#148] -(148) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_cdemo_sk#131] -Right keys [1]: [cd_demo_sk#142] +(148) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_cdemo_sk#136] +Right keys [1]: [cd_demo_sk#147] Join type: Inner -Join condition: NOT (cd_marital_status#141 = cd_marital_status#143) +Join condition: NOT (cd_marital_status#146 = cd_marital_status#148) -(149) Project [codegen id : 33] -Output [14]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [18]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141, cd_demo_sk#142, cd_marital_status#143] +(149) Project [codegen id : 49] +Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146, cd_demo_sk#147, cd_marital_status#148] -(150) ReusedExchange [Reuses operator id: 68] -Output [1]: [p_promo_sk#144] +(150) ReusedExchange [Reuses operator id: 70] +Output [1]: [p_promo_sk#149] -(151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_promo_sk#107] -Right keys [1]: [p_promo_sk#144] +(151) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_promo_sk#112] +Right keys [1]: [p_promo_sk#149] Join type: Inner Join condition: None -(152) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, p_promo_sk#144] +(152) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, p_promo_sk#149] -(153) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#145, hd_income_band_sk#146] +(153) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#150, hd_income_band_sk#151] -(154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#104] -Right keys [1]: [hd_demo_sk#145] +(154) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_hdemo_sk#109] +Right keys [1]: [hd_demo_sk#150] Join type: Inner Join condition: None -(155) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146] -Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_demo_sk#145, hd_income_band_sk#146] +(155) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_demo_sk#150, hd_income_band_sk#151] -(156) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] +(156) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] -(157) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_hdemo_sk#132] -Right keys [1]: [hd_demo_sk#147] +(157) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_hdemo_sk#137] +Right keys [1]: [hd_demo_sk#152] Join type: Inner Join condition: None -(158) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148] -Input [15]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_demo_sk#147, hd_income_band_sk#148] +(158) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153] +Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_demo_sk#152, hd_income_band_sk#153] -(159) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] +(159) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(160) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_addr_sk#105] -Right keys [1]: [ca_address_sk#149] +(160) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_addr_sk#110] +Right keys [1]: [ca_address_sk#154] Join type: Inner Join condition: None -(161) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -Input [18]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] +(161) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(162) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(162) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -(163) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_addr_sk#133] -Right keys [1]: [ca_address_sk#154] +(163) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_addr_sk#138] +Right keys [1]: [ca_address_sk#159] Join type: Inner Join condition: None -(164) Project [codegen id : 33] -Output [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [21]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(164) Project [codegen id : 49] +Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -(165) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#159] +(165) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#164] -(166) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [hd_income_band_sk#146] -Right keys [1]: [ib_income_band_sk#159] +(166) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#151] +Right keys [1]: [ib_income_band_sk#164] Join type: Inner Join condition: None -(167) Project [codegen id : 33] -Output [18]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [20]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#159] +(167) Project [codegen id : 49] +Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#164] -(168) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#160] +(168) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#165] -(169) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [hd_income_band_sk#148] -Right keys [1]: [ib_income_band_sk#160] +(169) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#153] +Right keys [1]: [ib_income_band_sk#165] Join type: Inner Join condition: None -(170) Project [codegen id : 33] -Output [17]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#160] +(170) Project [codegen id : 49] +Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#165] -(171) ReusedExchange [Reuses operator id: 102] -Output [2]: [i_item_sk#161, i_product_name#162] +(171) ReusedExchange [Reuses operator id: 104] +Output [2]: [i_item_sk#166, i_product_name#167] -(172) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_item_sk#101] -Right keys [1]: [i_item_sk#161] +(172) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [i_item_sk#166] Join type: Inner Join condition: None -(173) Project [codegen id : 33] -Output [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] - -(174) HashAggregate [codegen id : 33] -Input [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#109)), partial_sum(UnscaledValue(ss_list_price#110)), partial_sum(UnscaledValue(ss_coupon_amt#111))] -Aggregate Attributes [4]: [count#72, sum#163, sum#164, sum#165] -Results [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] - -(175) HashAggregate [codegen id : 33] -Input [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] -Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#109)), sum(UnscaledValue(ss_list_price#110)), sum(UnscaledValue(ss_coupon_amt#111))] -Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#109))#81, sum(UnscaledValue(ss_list_price#110))#82, sum(UnscaledValue(ss_coupon_amt#111))#83] -Results [8]: [i_item_sk#161 AS item_sk#169, s_store_name#128 AS store_name#170, s_zip#129 AS store_zip#171, d_year#126 AS syear#172, count(1)#80 AS cnt#173, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#109))#81,17,2) AS s1#174, MakeDecimal(sum(UnscaledValue(ss_list_price#110))#82,17,2) AS s2#175, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#111))#83,17,2) AS s3#176] - -(176) RowToColumnar -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] - -(177) CometColumnarExchange -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(178) CometSort -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176], [item_sk#169 ASC NULLS FIRST, store_name#170 ASC NULLS FIRST, store_zip#171 ASC NULLS FIRST] - -(179) ColumnarToRow [codegen id : 34] -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] - -(180) SortMergeJoin [codegen id : 35] -Left keys [3]: [item_sk#85, store_name#86, store_zip#87] -Right keys [3]: [item_sk#169, store_name#170, store_zip#171] +(173) Project [codegen id : 49] +Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] + +(174) HashAggregate [codegen id : 49] +Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] +Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count#77, sum#168, sum#169, sum#170] +Results [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] + +(175) HashAggregate [codegen id : 49] +Input [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] +Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] +Results [8]: [i_item_sk#166 AS item_sk#174, s_store_name#133 AS store_name#175, s_zip#134 AS store_zip#176, d_year#131 AS syear#177, count(1)#85 AS cnt#178, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#179, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#180, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#181] + +(176) Exchange +Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +Arguments: hashpartitioning(item_sk#174, store_name#175, store_zip#176, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(177) Sort [codegen id : 50] +Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +Arguments: [item_sk#174 ASC NULLS FIRST, store_name#175 ASC NULLS FIRST, store_zip#176 ASC NULLS FIRST], false, 0 + +(178) SortMergeJoin [codegen id : 51] +Left keys [3]: [item_sk#90, store_name#91, store_zip#92] +Right keys [3]: [item_sk#174, store_name#175, store_zip#176] Join type: Inner -Join condition: (cnt#173 <= cnt#97) - -(181) Project [codegen id : 35] -Output [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Input [25]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] +Join condition: (cnt#178 <= cnt#102) -(182) RowToColumnar -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] +(179) Project [codegen id : 51] +Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -(183) CometColumnarExchange -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, s1#98 ASC NULLS FIRST, s1#174 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +(180) Exchange +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#179 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] -(184) CometSort -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173], [product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, s1#98 ASC NULLS FIRST, s1#174 ASC NULLS FIRST] - -(185) ColumnarToRow [codegen id : 36] -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] +(181) Sort [codegen id : 52] +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#179 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (189) -+- * ColumnarToRow (188) - +- CometFilter (187) - +- CometScan parquet spark_catalog.default.date_dim (186) +BroadcastExchange (185) ++- * ColumnarToRow (184) + +- CometFilter (183) + +- CometScan parquet spark_catalog.default.date_dim (182) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] +Output [2]: [d_date_sk#37, d_year#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(187) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) +(183) CometFilter +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) -(188) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] +(184) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#37, d_year#38] -(189) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] +(185) BroadcastExchange +Input [2]: [d_date_sk#37, d_year#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 -BroadcastExchange (193) -+- * ColumnarToRow (192) - +- CometFilter (191) - +- CometScan parquet spark_catalog.default.date_dim (190) +Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (189) ++- * ColumnarToRow (188) + +- CometFilter (187) + +- CometScan parquet spark_catalog.default.date_dim (186) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#125, d_year#126] +Output [2]: [d_date_sk#130, d_year#131] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(191) CometFilter -Input [2]: [d_date_sk#125, d_year#126] -Condition : ((isnotnull(d_year#126) AND (d_year#126 = 2000)) AND isnotnull(d_date_sk#125)) +(187) CometFilter +Input [2]: [d_date_sk#130, d_year#131] +Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) -(192) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#125, d_year#126] +(188) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#130, d_year#131] -(193) BroadcastExchange -Input [2]: [d_date_sk#125, d_year#126] +(189) BroadcastExchange +Input [2]: [d_date_sk#130, d_year#131] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index f297b41473..2a0bc5bced 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -1,58 +1,58 @@ -WholeStageCodegen (36) - ColumnarToRow +WholeStageCodegen (52) + Sort [product_name,store_name,cnt,s1,s1] InputAdapter - CometSort [product_name,store_name,cnt,s1,s1] - CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 - RowToColumnar - WholeStageCodegen (35) - Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] - InputAdapter - WholeStageCodegen (17) - ColumnarToRow - InputAdapter - CometSort [item_sk,store_name,store_zip] - CometColumnarExchange [item_sk,store_name,store_zip] #2 - RowToColumnar - WholeStageCodegen (16) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow + Exchange [product_name,store_name,cnt,s1,s1] #1 + WholeStageCodegen (51) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (25) + Sort [item_sk,store_name,store_zip] + InputAdapter + Exchange [item_sk,store_name,store_zip] #2 + WholeStageCodegen (24) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometColumnarExchange [ss_item_sk] #3 + Exchange [ss_item_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] CometBroadcastExchange #4 @@ -68,146 +68,164 @@ WholeStageCodegen (36) CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - CometColumnarExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometColumnarExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number] - CometColumnarExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + WholeStageCodegen (8) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #6 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (4) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #8 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (4) + BroadcastExchange #9 + WholeStageCodegen (10) ColumnarToRow InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + BroadcastExchange #10 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (6) + BroadcastExchange #11 + WholeStageCodegen (12) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) + BroadcastExchange #12 + WholeStageCodegen (14) ColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - BroadcastExchange #14 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + BroadcastExchange #13 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) + BroadcastExchange #14 + WholeStageCodegen (17) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (13) + BroadcastExchange #15 + WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter - ReusedExchange [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - BroadcastExchange #17 - WholeStageCodegen (15) + BroadcastExchange #16 + WholeStageCodegen (21) ColumnarToRow InputAdapter - CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - InputAdapter - WholeStageCodegen (34) - ColumnarToRow - InputAdapter - CometSort [item_sk,store_name,store_zip] - CometColumnarExchange [item_sk,store_name,store_zip] #18 - RowToColumnar - WholeStageCodegen (33) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (23) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_product_name] + CometFilter [i_current_price,i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (50) + Sort [item_sk,store_name,store_zip] + InputAdapter + Exchange [item_sk,store_name,store_zip] #18 + WholeStageCodegen (49) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (27) + Sort [ss_item_sk] InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometColumnarExchange [ss_item_sk] #19 + Exchange [ss_item_sk] #19 + WholeStageCodegen (26) + ColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] CometBroadcastExchange #20 @@ -223,38 +241,41 @@ WholeStageCodegen (36) CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #21 InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + WholeStageCodegen (33) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 + ReusedExchange [d_date_sk,d_year] #21 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [p_promo_sk] #13 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [p_promo_sk] #13 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [i_item_sk,i_product_name] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index 951fab3e84..68be5c8723 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -1,99 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (95) -+- * Filter (94) - +- Window (93) - +- * ColumnarToRow (92) - +- CometSort (91) - +- CometColumnarExchange (90) - +- RowToColumnar (89) - +- Union (88) - :- * HashAggregate (23) - : +- * ColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- RowToColumnar (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.item (13) - :- * HashAggregate (31) - : +- * ColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- RowToColumnar (28) - : +- * HashAggregate (27) - : +- * HashAggregate (26) - : +- * ColumnarToRow (25) - : +- ReusedExchange (24) - :- * HashAggregate (39) - : +- * ColumnarToRow (38) - : +- CometColumnarExchange (37) - : +- RowToColumnar (36) - : +- * HashAggregate (35) - : +- * HashAggregate (34) - : +- * ColumnarToRow (33) - : +- ReusedExchange (32) - :- * HashAggregate (47) - : +- * ColumnarToRow (46) - : +- CometColumnarExchange (45) - : +- RowToColumnar (44) - : +- * HashAggregate (43) - : +- * HashAggregate (42) - : +- * ColumnarToRow (41) - : +- ReusedExchange (40) - :- * HashAggregate (55) - : +- * ColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- RowToColumnar (52) - : +- * HashAggregate (51) - : +- * HashAggregate (50) - : +- * ColumnarToRow (49) - : +- ReusedExchange (48) - :- * HashAggregate (63) - : +- * ColumnarToRow (62) - : +- CometColumnarExchange (61) - : +- RowToColumnar (60) - : +- * HashAggregate (59) - : +- * HashAggregate (58) - : +- * ColumnarToRow (57) - : +- ReusedExchange (56) - :- * HashAggregate (71) - : +- * ColumnarToRow (70) - : +- CometColumnarExchange (69) - : +- RowToColumnar (68) - : +- * HashAggregate (67) - : +- * HashAggregate (66) - : +- * ColumnarToRow (65) - : +- ReusedExchange (64) - :- * HashAggregate (79) - : +- * ColumnarToRow (78) - : +- CometColumnarExchange (77) - : +- RowToColumnar (76) - : +- * HashAggregate (75) - : +- * HashAggregate (74) - : +- * ColumnarToRow (73) - : +- ReusedExchange (72) - +- * HashAggregate (87) - +- * ColumnarToRow (86) - +- CometColumnarExchange (85) - +- RowToColumnar (84) - +- * HashAggregate (83) - +- * HashAggregate (82) - +- * ColumnarToRow (81) - +- ReusedExchange (80) +TakeOrderedAndProject (67) ++- * Filter (66) + +- Window (65) + +- * Sort (64) + +- Exchange (63) + +- Union (62) + :- * HashAggregate (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.item (13) + :- * HashAggregate (26) + : +- Exchange (25) + : +- * HashAggregate (24) + : +- * HashAggregate (23) + : +- ReusedExchange (22) + :- * HashAggregate (31) + : +- Exchange (30) + : +- * HashAggregate (29) + : +- * HashAggregate (28) + : +- ReusedExchange (27) + :- * HashAggregate (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- * HashAggregate (33) + : +- ReusedExchange (32) + :- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * HashAggregate (38) + : +- ReusedExchange (37) + :- * HashAggregate (46) + : +- Exchange (45) + : +- * HashAggregate (44) + : +- * HashAggregate (43) + : +- ReusedExchange (42) + :- * HashAggregate (51) + : +- Exchange (50) + : +- * HashAggregate (49) + : +- * HashAggregate (48) + : +- ReusedExchange (47) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * HashAggregate (53) + : +- ReusedExchange (52) + +- * HashAggregate (61) + +- Exchange (60) + +- * HashAggregate (59) + +- * HashAggregate (58) + +- ReusedExchange (57) (unknown) Scan parquet spark_catalog.default.store_sales @@ -111,7 +83,7 @@ Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (3) ColumnarToRow [codegen id : 4] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(4) ReusedExchange [Reuses operator id: 100] +(4) ReusedExchange [Reuses operator id: 72] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (5) BroadcastHashJoin [codegen id : 4] @@ -187,355 +159,271 @@ Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as d Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] -(20) RowToColumnar +(20) Exchange Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometColumnarExchange -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) ColumnarToRow [codegen id : 5] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] - -(23) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 5] Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23] -(24) ReusedExchange [Reuses operator id: 21] +(22) ReusedExchange [Reuses operator id: 20] Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25] -(25) ColumnarToRow [codegen id : 10] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25] - -(26) HashAggregate [codegen id : 10] +(23) HashAggregate [codegen id : 10] Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(27) HashAggregate [codegen id : 10] +(24) HashAggregate [codegen id : 10] Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sumsales#26] Keys [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9] Functions [1]: [partial_sum(sumsales#26)] Aggregate Attributes [2]: [sum#27, isEmpty#28] Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] -(28) RowToColumnar -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] - -(29) CometColumnarExchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) ColumnarToRow [codegen id : 11] +(25) Exchange Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(31) HashAggregate [codegen id : 11] +(26) HashAggregate [codegen id : 11] Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] Keys [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9] Functions [1]: [sum(sumsales#26)] Aggregate Attributes [1]: [sum(sumsales#26)#31] Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, null AS s_store_id#32, sum(sumsales#26)#31 AS sumsales#33] -(32) ReusedExchange [Reuses operator id: 21] +(27) ReusedExchange [Reuses operator id: 20] Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35] -(33) ColumnarToRow [codegen id : 16] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35] - -(34) HashAggregate [codegen id : 16] +(28) HashAggregate [codegen id : 16] Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(35) HashAggregate [codegen id : 16] +(29) HashAggregate [codegen id : 16] Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sumsales#26] Keys [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10] Functions [1]: [partial_sum(sumsales#26)] Aggregate Attributes [2]: [sum#36, isEmpty#37] Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] -(36) RowToColumnar -Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] - -(37) CometColumnarExchange +(30) Exchange Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) ColumnarToRow [codegen id : 17] -Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] - -(39) HashAggregate [codegen id : 17] +(31) HashAggregate [codegen id : 17] Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] Keys [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10] Functions [1]: [sum(sumsales#26)] Aggregate Attributes [1]: [sum(sumsales#26)#40] Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, null AS d_moy#41, null AS s_store_id#42, sum(sumsales#26)#40 AS sumsales#43] -(40) ReusedExchange [Reuses operator id: 21] +(32) ReusedExchange [Reuses operator id: 20] Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45] -(41) ColumnarToRow [codegen id : 22] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45] - -(42) HashAggregate [codegen id : 22] +(33) HashAggregate [codegen id : 22] Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(43) HashAggregate [codegen id : 22] +(34) HashAggregate [codegen id : 22] Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sumsales#26] Keys [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8] Functions [1]: [partial_sum(sumsales#26)] Aggregate Attributes [2]: [sum#46, isEmpty#47] Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] -(44) RowToColumnar -Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] - -(45) CometColumnarExchange -Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) ColumnarToRow [codegen id : 23] +(35) Exchange Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(47) HashAggregate [codegen id : 23] +(36) HashAggregate [codegen id : 23] Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] Keys [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8] Functions [1]: [sum(sumsales#26)] Aggregate Attributes [1]: [sum(sumsales#26)#50] Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, null AS d_qoy#51, null AS d_moy#52, null AS s_store_id#53, sum(sumsales#26)#50 AS sumsales#54] -(48) ReusedExchange [Reuses operator id: 21] +(37) ReusedExchange [Reuses operator id: 20] Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56] -(49) ColumnarToRow [codegen id : 28] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56] - -(50) HashAggregate [codegen id : 28] +(38) HashAggregate [codegen id : 28] Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] Results [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(51) HashAggregate [codegen id : 28] +(39) HashAggregate [codegen id : 28] Input [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sumsales#26] Keys [4]: [i_category#16, i_class#15, i_brand#14, i_product_name#17] Functions [1]: [partial_sum(sumsales#26)] Aggregate Attributes [2]: [sum#57, isEmpty#58] Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] -(52) RowToColumnar -Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] - -(53) CometColumnarExchange -Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(54) ColumnarToRow [codegen id : 29] +(40) Exchange Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(55) HashAggregate [codegen id : 29] +(41) HashAggregate [codegen id : 29] Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] Keys [4]: [i_category#16, i_class#15, i_brand#14, i_product_name#17] Functions [1]: [sum(sumsales#26)] Aggregate Attributes [1]: [sum(sumsales#26)#61] Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, null AS d_year#62, null AS d_qoy#63, null AS d_moy#64, null AS s_store_id#65, sum(sumsales#26)#61 AS sumsales#66] -(56) ReusedExchange [Reuses operator id: 21] +(42) ReusedExchange [Reuses operator id: 20] Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68] -(57) ColumnarToRow [codegen id : 34] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68] - -(58) HashAggregate [codegen id : 34] +(43) HashAggregate [codegen id : 34] Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] Results [4]: [i_category#16, i_class#15, i_brand#14, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(59) HashAggregate [codegen id : 34] +(44) HashAggregate [codegen id : 34] Input [4]: [i_category#16, i_class#15, i_brand#14, sumsales#26] Keys [3]: [i_category#16, i_class#15, i_brand#14] Functions [1]: [partial_sum(sumsales#26)] Aggregate Attributes [2]: [sum#69, isEmpty#70] Results [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] -(60) RowToColumnar -Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] - -(61) CometColumnarExchange +(45) Exchange Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(62) ColumnarToRow [codegen id : 35] -Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] - -(63) HashAggregate [codegen id : 35] +(46) HashAggregate [codegen id : 35] Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] Keys [3]: [i_category#16, i_class#15, i_brand#14] Functions [1]: [sum(sumsales#26)] Aggregate Attributes [1]: [sum(sumsales#26)#73] Results [9]: [i_category#16, i_class#15, i_brand#14, null AS i_product_name#74, null AS d_year#75, null AS d_qoy#76, null AS d_moy#77, null AS s_store_id#78, sum(sumsales#26)#73 AS sumsales#79] -(64) ReusedExchange [Reuses operator id: 21] +(47) ReusedExchange [Reuses operator id: 20] Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81] -(65) ColumnarToRow [codegen id : 40] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81] - -(66) HashAggregate [codegen id : 40] +(48) HashAggregate [codegen id : 40] Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] Results [3]: [i_category#16, i_class#15, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(67) HashAggregate [codegen id : 40] +(49) HashAggregate [codegen id : 40] Input [3]: [i_category#16, i_class#15, sumsales#26] Keys [2]: [i_category#16, i_class#15] Functions [1]: [partial_sum(sumsales#26)] Aggregate Attributes [2]: [sum#82, isEmpty#83] Results [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] -(68) RowToColumnar -Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] - -(69) CometColumnarExchange +(50) Exchange Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] -Arguments: hashpartitioning(i_category#16, i_class#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Arguments: hashpartitioning(i_category#16, i_class#15, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(70) ColumnarToRow [codegen id : 41] -Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] - -(71) HashAggregate [codegen id : 41] +(51) HashAggregate [codegen id : 41] Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] Keys [2]: [i_category#16, i_class#15] Functions [1]: [sum(sumsales#26)] Aggregate Attributes [1]: [sum(sumsales#26)#86] Results [9]: [i_category#16, i_class#15, null AS i_brand#87, null AS i_product_name#88, null AS d_year#89, null AS d_qoy#90, null AS d_moy#91, null AS s_store_id#92, sum(sumsales#26)#86 AS sumsales#93] -(72) ReusedExchange [Reuses operator id: 21] +(52) ReusedExchange [Reuses operator id: 20] Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95] -(73) ColumnarToRow [codegen id : 46] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95] - -(74) HashAggregate [codegen id : 46] +(53) HashAggregate [codegen id : 46] Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] Results [2]: [i_category#16, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(75) HashAggregate [codegen id : 46] +(54) HashAggregate [codegen id : 46] Input [2]: [i_category#16, sumsales#26] Keys [1]: [i_category#16] Functions [1]: [partial_sum(sumsales#26)] Aggregate Attributes [2]: [sum#96, isEmpty#97] Results [3]: [i_category#16, sum#98, isEmpty#99] -(76) RowToColumnar +(55) Exchange Input [3]: [i_category#16, sum#98, isEmpty#99] +Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(77) CometColumnarExchange -Input [3]: [i_category#16, sum#98, isEmpty#99] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(78) ColumnarToRow [codegen id : 47] -Input [3]: [i_category#16, sum#98, isEmpty#99] - -(79) HashAggregate [codegen id : 47] +(56) HashAggregate [codegen id : 47] Input [3]: [i_category#16, sum#98, isEmpty#99] Keys [1]: [i_category#16] Functions [1]: [sum(sumsales#26)] Aggregate Attributes [1]: [sum(sumsales#26)#100] Results [9]: [i_category#16, null AS i_class#101, null AS i_brand#102, null AS i_product_name#103, null AS d_year#104, null AS d_qoy#105, null AS d_moy#106, null AS s_store_id#107, sum(sumsales#26)#100 AS sumsales#108] -(80) ReusedExchange [Reuses operator id: 21] +(57) ReusedExchange [Reuses operator id: 20] Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110] -(81) ColumnarToRow [codegen id : 52] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110] - -(82) HashAggregate [codegen id : 52] +(58) HashAggregate [codegen id : 52] Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110] Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] Results [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] -(83) HashAggregate [codegen id : 52] +(59) HashAggregate [codegen id : 52] Input [1]: [sumsales#26] Keys: [] Functions [1]: [partial_sum(sumsales#26)] Aggregate Attributes [2]: [sum#111, isEmpty#112] Results [2]: [sum#113, isEmpty#114] -(84) RowToColumnar -Input [2]: [sum#113, isEmpty#114] - -(85) CometColumnarExchange -Input [2]: [sum#113, isEmpty#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(86) ColumnarToRow [codegen id : 53] +(60) Exchange Input [2]: [sum#113, isEmpty#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] -(87) HashAggregate [codegen id : 53] +(61) HashAggregate [codegen id : 53] Input [2]: [sum#113, isEmpty#114] Keys: [] Functions [1]: [sum(sumsales#26)] Aggregate Attributes [1]: [sum(sumsales#26)#115] Results [9]: [null AS i_category#116, null AS i_class#117, null AS i_brand#118, null AS i_product_name#119, null AS d_year#120, null AS d_qoy#121, null AS d_moy#122, null AS s_store_id#123, sum(sumsales#26)#115 AS sumsales#124] -(88) Union - -(89) RowToColumnar -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] - -(90) CometColumnarExchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +(62) Union -(91) CometSort +(63) Exchange Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23], [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST] +Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(92) ColumnarToRow [codegen id : 54] +(64) Sort [codegen id : 54] Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] +Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 -(93) Window +(65) Window Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#125], [i_category#16], [sumsales#23 DESC NULLS LAST] -(94) Filter [codegen id : 55] +(66) Filter [codegen id : 55] Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] Condition : (rk#125 <= 100) -(95) TakeOrderedAndProject +(67) TakeOrderedAndProject Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#125 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (100) -+- * ColumnarToRow (99) - +- CometProject (98) - +- CometFilter (97) - +- CometScan parquet spark_catalog.default.date_dim (96) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) (unknown) Scan parquet spark_catalog.default.date_dim @@ -545,18 +433,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(97) CometFilter +(69) CometFilter Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10] Condition : (((isnotnull(d_month_seq#126) AND (d_month_seq#126 >= 1212)) AND (d_month_seq#126 <= 1223)) AND isnotnull(d_date_sk#7)) -(98) CometProject +(70) CometProject Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(99) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(100) BroadcastExchange +(72) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index 72f33d82e8..cfac29f8a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -4,147 +4,119 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Window [sumsales,i_category] WholeStageCodegen (54) - ColumnarToRow + Sort [i_category,sumsales] InputAdapter - CometSort [i_category,sumsales] - CometColumnarExchange [i_category] #1 - RowToColumnar - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - RowToColumnar - WholeStageCodegen (16) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - RowToColumnar - WholeStageCodegen (22) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) - HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 - RowToColumnar - WholeStageCodegen (28) - HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) - HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand] #10 - RowToColumnar - WholeStageCodegen (34) - HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) - HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #11 - RowToColumnar - WholeStageCodegen (40) - HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) - HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #12 - RowToColumnar - WholeStageCodegen (46) - HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - ColumnarToRow + Exchange [i_category] #1 + Union + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange #13 - RowToColumnar - WholeStageCodegen (52) - HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + WholeStageCodegen (11) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 + WholeStageCodegen (10) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (17) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 + WholeStageCodegen (16) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (23) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name,d_year] #8 + WholeStageCodegen (22) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (29) + HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name] #9 + WholeStageCodegen (28) + HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (35) + HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand] #10 + WholeStageCodegen (34) + HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (41) + HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class] #11 + WholeStageCodegen (40) + HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (47) + HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category] #12 + WholeStageCodegen (46) + HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (53) + HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange #13 + WholeStageCodegen (52) + HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index ffae6c6604..77dcb698fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -1,71 +1,59 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- Window (65) - +- * ColumnarToRow (64) - +- CometSort (63) - +- CometColumnarExchange (62) - +- CometHashAggregate (61) - +- CometColumnarExchange (60) - +- RowToColumnar (59) - +- * HashAggregate (58) - +- Union (57) - :- * HashAggregate (40) - : +- * ColumnarToRow (39) - : +- CometColumnarExchange (38) - : +- RowToColumnar (37) - : +- * HashAggregate (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (33) - : +- * BroadcastHashJoin LeftSemi BuildRight (32) - : :- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- Window (28) - : +- * Sort (27) - : +- * HashAggregate (26) - : +- * ColumnarToRow (25) - : +- CometColumnarExchange (24) - : +- RowToColumnar (23) - : +- * HashAggregate (22) - : +- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * ColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.store_sales (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- ReusedExchange (19) - :- * HashAggregate (48) - : +- * ColumnarToRow (47) - : +- CometColumnarExchange (46) - : +- RowToColumnar (45) - : +- * HashAggregate (44) - : +- * HashAggregate (43) - : +- * ColumnarToRow (42) - : +- ReusedExchange (41) - +- * HashAggregate (56) - +- * ColumnarToRow (55) - +- CometColumnarExchange (54) - +- RowToColumnar (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * ColumnarToRow (50) - +- ReusedExchange (49) +TakeOrderedAndProject (55) ++- * Project (54) + +- Window (53) + +- * Sort (52) + +- Exchange (51) + +- * HashAggregate (50) + +- Exchange (49) + +- * HashAggregate (48) + +- Union (47) + :- * HashAggregate (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (31) + : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : :- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (29) + : +- * Project (28) + : +- * Filter (27) + : +- Window (26) + : +- * Sort (25) + : +- * HashAggregate (24) + : +- Exchange (23) + : +- * HashAggregate (22) + : +- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * ColumnarToRow (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.store_sales (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- ReusedExchange (19) + :- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * HashAggregate (38) + : +- ReusedExchange (37) + +- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- ReusedExchange (42) (unknown) Scan parquet spark_catalog.default.store_sales @@ -83,7 +71,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 8] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 72] +(4) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 8] @@ -153,7 +141,7 @@ Join condition: None Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] -(19) ReusedExchange [Reuses operator id: 72] +(19) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#15] (20) BroadcastHashJoin [codegen id : 4] @@ -173,213 +161,179 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum#16] Results [2]: [s_state#14, sum#17] -(23) RowToColumnar +(23) Exchange Input [2]: [s_state#14, sum#17] +Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(24) CometColumnarExchange -Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(25) ColumnarToRow [codegen id : 5] -Input [2]: [s_state#14, sum#17] - -(26) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 5] Input [2]: [s_state#14, sum#17] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] -(27) Sort [codegen id : 5] +(25) Sort [codegen id : 5] Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 -(28) Window +(26) Window Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] -(29) Filter [codegen id : 6] +(27) Filter [codegen id : 6] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Condition : (ranking#20 <= 5) -(30) Project [codegen id : 6] +(28) Project [codegen id : 6] Output [1]: [s_state#14] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] -(31) BroadcastExchange +(29) BroadcastExchange Input [1]: [s_state#14] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(32) BroadcastHashJoin [codegen id : 7] +(30) BroadcastHashJoin [codegen id : 7] Left keys [1]: [s_state#8] Right keys [1]: [s_state#14] Join type: LeftSemi Join condition: None -(33) BroadcastExchange +(31) BroadcastExchange Input [3]: [s_store_sk#6, s_county#7, s_state#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(35) Project [codegen id : 8] +(33) Project [codegen id : 8] Output [3]: [ss_net_profit#2, s_county#7, s_state#8] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] -(36) HashAggregate [codegen id : 8] +(34) HashAggregate [codegen id : 8] Input [3]: [ss_net_profit#2, s_county#7, s_state#8] Keys [2]: [s_state#8, s_county#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#21] Results [3]: [s_state#8, s_county#7, sum#22] -(37) RowToColumnar -Input [3]: [s_state#8, s_county#7, sum#22] - -(38) CometColumnarExchange -Input [3]: [s_state#8, s_county#7, sum#22] -Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(39) ColumnarToRow [codegen id : 9] +(35) Exchange Input [3]: [s_state#8, s_county#7, sum#22] +Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(40) HashAggregate [codegen id : 9] +(36) HashAggregate [codegen id : 9] Input [3]: [s_state#8, s_county#7, sum#22] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) as decimal(27,2)) AS total_sum#24, s_state#8, s_county#7, 0 AS g_state#25, 0 AS g_county#26, 0 AS lochierarchy#27] -(41) ReusedExchange [Reuses operator id: 38] +(37) ReusedExchange [Reuses operator id: 35] Output [3]: [s_state#8, s_county#7, sum#28] -(42) ColumnarToRow [codegen id : 18] -Input [3]: [s_state#8, s_county#7, sum#28] - -(43) HashAggregate [codegen id : 18] +(38) HashAggregate [codegen id : 18] Input [3]: [s_state#8, s_county#7, sum#28] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS total_sum#29, s_state#8] -(44) HashAggregate [codegen id : 18] +(39) HashAggregate [codegen id : 18] Input [2]: [total_sum#29, s_state#8] Keys [1]: [s_state#8] Functions [1]: [partial_sum(total_sum#29)] Aggregate Attributes [2]: [sum#30, isEmpty#31] Results [3]: [s_state#8, sum#32, isEmpty#33] -(45) RowToColumnar -Input [3]: [s_state#8, sum#32, isEmpty#33] - -(46) CometColumnarExchange +(40) Exchange Input [3]: [s_state#8, sum#32, isEmpty#33] -Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(47) ColumnarToRow [codegen id : 19] -Input [3]: [s_state#8, sum#32, isEmpty#33] - -(48) HashAggregate [codegen id : 19] +(41) HashAggregate [codegen id : 19] Input [3]: [s_state#8, sum#32, isEmpty#33] Keys [1]: [s_state#8] Functions [1]: [sum(total_sum#29)] Aggregate Attributes [1]: [sum(total_sum#29)#34] Results [6]: [sum(total_sum#29)#34 AS total_sum#35, s_state#8, null AS s_county#36, 0 AS g_state#37, 1 AS g_county#38, 1 AS lochierarchy#39] -(49) ReusedExchange [Reuses operator id: 38] +(42) ReusedExchange [Reuses operator id: 35] Output [3]: [s_state#8, s_county#7, sum#40] -(50) ColumnarToRow [codegen id : 28] -Input [3]: [s_state#8, s_county#7, sum#40] - -(51) HashAggregate [codegen id : 28] +(43) HashAggregate [codegen id : 28] Input [3]: [s_state#8, s_county#7, sum#40] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS total_sum#29] -(52) HashAggregate [codegen id : 28] +(44) HashAggregate [codegen id : 28] Input [1]: [total_sum#29] Keys: [] Functions [1]: [partial_sum(total_sum#29)] Aggregate Attributes [2]: [sum#41, isEmpty#42] Results [2]: [sum#43, isEmpty#44] -(53) RowToColumnar +(45) Exchange Input [2]: [sum#43, isEmpty#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(54) CometColumnarExchange -Input [2]: [sum#43, isEmpty#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(55) ColumnarToRow [codegen id : 29] -Input [2]: [sum#43, isEmpty#44] - -(56) HashAggregate [codegen id : 29] +(46) HashAggregate [codegen id : 29] Input [2]: [sum#43, isEmpty#44] Keys: [] Functions [1]: [sum(total_sum#29)] Aggregate Attributes [1]: [sum(total_sum#29)#45] Results [6]: [sum(total_sum#29)#45 AS total_sum#46, null AS s_state#47, null AS s_county#48, 1 AS g_state#49, 1 AS g_county#50, 2 AS lochierarchy#51] -(57) Union +(47) Union -(58) HashAggregate [codegen id : 30] +(48) HashAggregate [codegen id : 30] Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] Results [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -(59) RowToColumnar -Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] - -(60) CometColumnarExchange +(49) Exchange Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -Arguments: hashpartitioning(total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: hashpartitioning(total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(61) CometHashAggregate +(50) HashAggregate [codegen id : 31] Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] Functions: [] +Aggregate Attributes: [] +Results [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, CASE WHEN (g_county#26 = 0) THEN s_state#8 END AS _w0#52] -(62) CometColumnarExchange -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] -Arguments: hashpartitioning(lochierarchy#27, _w0#52, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(63) CometSort +(51) Exchange Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] -Arguments: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52], [lochierarchy#27 ASC NULLS FIRST, _w0#52 ASC NULLS FIRST, total_sum#24 DESC NULLS LAST] +Arguments: hashpartitioning(lochierarchy#27, _w0#52, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(64) ColumnarToRow [codegen id : 31] +(52) Sort [codegen id : 32] Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#52 ASC NULLS FIRST, total_sum#24 DESC NULLS LAST], false, 0 -(65) Window +(53) Window Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] Arguments: [rank(total_sum#24) windowspecdefinition(lochierarchy#27, _w0#52, total_sum#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#53], [lochierarchy#27, _w0#52], [total_sum#24 DESC NULLS LAST] -(66) Project [codegen id : 32] +(54) Project [codegen id : 33] Output [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] Input [6]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52, rank_within_parent#53] -(67) TakeOrderedAndProject +(55) TakeOrderedAndProject Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#53 ASC NULLS FIRST], [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (60) ++- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) (unknown) Scan parquet spark_catalog.default.date_dim @@ -389,18 +343,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter +(57) CometFilter Input [2]: [d_date_sk#5, d_month_seq#54] Condition : (((isnotnull(d_month_seq#54) AND (d_month_seq#54 >= 1212)) AND (d_month_seq#54 <= 1223)) AND isnotnull(d_date_sk#5)) -(70) CometProject +(58) CometProject Input [2]: [d_date_sk#5, d_month_seq#54] Arguments: [d_date_sk#5], [d_date_sk#5] -(71) ColumnarToRow [codegen id : 1] +(59) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(72) BroadcastExchange +(60) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 1a6112bed1..663f828f46 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -1,109 +1,99 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (32) + WholeStageCodegen (33) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (31) - ColumnarToRow + WholeStageCodegen (32) + Sort [lochierarchy,_w0,total_sum] InputAdapter - CometSort [lochierarchy,_w0,total_sum] - CometColumnarExchange [lochierarchy,_w0] #1 - CometHashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - RowToColumnar + Exchange [lochierarchy,_w0] #1 + WholeStageCodegen (31) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] [_w0] + InputAdapter + Exchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 WholeStageCodegen (30) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union WholeStageCodegen (9) HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + InputAdapter + Exchange [s_state,s_county] #3 + WholeStageCodegen (8) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - BroadcastHashJoin [s_state,s_state] - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #7 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (6) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WholeStageCodegen (5) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + Exchange [s_state] #7 + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #4 WholeStageCodegen (19) HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #9 - RowToColumnar - WholeStageCodegen (18) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - ColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + InputAdapter + Exchange [s_state] #9 + WholeStageCodegen (18) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 WholeStageCodegen (29) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange #10 - RowToColumnar - WholeStageCodegen (28) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - ColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + InputAdapter + Exchange #10 + WholeStageCodegen (28) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index 9fe23081f5..00c2dabde4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -1,75 +1,74 @@ == Physical Plan == -* ColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometHashAggregate (69) - +- CometColumnarExchange (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometSortMergeJoin (65) - :- CometSort (59) - : +- CometColumnarExchange (58) - : +- RowToColumnar (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * ColumnarToRow (3) - : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * ColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * ColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * ColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * ColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * ColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * ColumnarToRow (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.promotion (51) - +- CometSort (64) - +- CometColumnarExchange (63) - +- CometProject (62) - +- CometFilter (61) - +- CometScan parquet spark_catalog.default.catalog_returns (60) +TakeOrderedAndProject (70) ++- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * Project (66) + +- * SortMergeJoin LeftOuter (65) + :- * Sort (58) + : +- Exchange (57) + : +- * Project (56) + : +- * BroadcastHashJoin LeftOuter BuildRight (55) + : :- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Project (15) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : : :- * Project (9) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : : : : :- * ColumnarToRow (3) + : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- BroadcastExchange (7) + : : : : : : : : : +- * ColumnarToRow (6) + : : : : : : : : : +- CometFilter (5) + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) + : : : : : : : : +- BroadcastExchange (13) + : : : : : : : : +- * ColumnarToRow (12) + : : : : : : : : +- CometFilter (11) + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * ColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- CometProject (24) + : : : : : : +- CometFilter (23) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) + : : : : : +- BroadcastExchange (33) + : : : : : +- * ColumnarToRow (32) + : : : : : +- CometProject (31) + : : : : : +- CometFilter (30) + : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) + : : : : +- ReusedExchange (36) + : : : +- BroadcastExchange (42) + : : : +- * ColumnarToRow (41) + : : : +- CometFilter (40) + : : : +- CometScan parquet spark_catalog.default.date_dim (39) + : : +- BroadcastExchange (48) + : : +- * ColumnarToRow (47) + : : +- CometFilter (46) + : : +- CometScan parquet spark_catalog.default.date_dim (45) + : +- BroadcastExchange (54) + : +- * ColumnarToRow (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.promotion (51) + +- * Sort (64) + +- Exchange (63) + +- * ColumnarToRow (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.catalog_returns (59) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -236,7 +235,7 @@ Join condition: None Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -(36) ReusedExchange [Reuses operator id: 76] +(36) ReusedExchange [Reuses operator id: 75] Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (37) BroadcastHashJoin [codegen id : 10] @@ -333,16 +332,13 @@ Join condition: None Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] -(57) RowToColumnar +(57) Exchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(58) CometColumnarExchange +(58) Sort [codegen id : 11] Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(59) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] +Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] @@ -351,81 +347,86 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(61) CometFilter +(60) CometFilter Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) -(62) CometProject +(61) CometProject Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] -(63) CometColumnarExchange +(62) ColumnarToRow [codegen id : 12] Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(64) CometSort +(63) Exchange Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(65) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter +(64) Sort [codegen id : 13] +Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST], false, 0 -(66) CometProject +(65) SortMergeJoin [codegen id : 14] +Left keys [2]: [cs_item_sk#4, cs_order_number#6] +Right keys [2]: [cr_item_sk#30, cr_order_number#31] +Join type: LeftOuter +Join condition: None + +(66) Project [codegen id : 14] +Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(67) CometHashAggregate +(67) HashAggregate [codegen id : 14] Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#33] +Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -(68) CometColumnarExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(68) Exchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(69) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] +(69) HashAggregate [codegen id : 15] +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#35] +Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#35 AS no_promo#36, count(1)#35 AS promo#37, count(1)#35 AS total_cnt#38] -(70) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), 100, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -(71) ColumnarToRow [codegen id : 11] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] +(70) TakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] +Arguments: 100, [total_cnt#38 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) (unknown) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(73) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(72) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +Condition : ((((isnotnull(d_year#39) AND (d_year#39 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(74) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +(73) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(75) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(76) BroadcastExchange +(75) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index 5bf7ab343a..17fc9dee7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -1,108 +1,116 @@ -WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometColumnarExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - RowToColumnar - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - ColumnarToRow - InputAdapter - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] +TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] + WholeStageCodegen (15) + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] + InputAdapter + Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + WholeStageCodegen (14) + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] + Project [w_warehouse_name,i_item_desc,d_week_seq] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (11) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #2 + WholeStageCodegen (10) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + ColumnarToRow + InputAdapter + CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) + BroadcastExchange #5 + WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + BroadcastExchange #6 + WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) + BroadcastExchange #7 + WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometProject [cd_demo_sk] + CometFilter [cd_marital_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) + BroadcastExchange #8 + WholeStageCodegen (5) ColumnarToRow InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + BroadcastExchange #9 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [d_week_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) + BroadcastExchange #10 + WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [d_week_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) + BroadcastExchange #11 + WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometColumnarExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + InputAdapter + WholeStageCodegen (13) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #12 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index bb7f608784..776fad0078 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -1,83 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * ColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- RowToColumnar (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * ColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- RowToColumnar (32) - : : +- * HashAggregate (31) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Project (27) - : : : +- * BroadcastHashJoin Inner BuildRight (26) - : : : :- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.store_sales (22) - : : +- ReusedExchange (28) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * ColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- RowToColumnar (51) - : +- * HashAggregate (50) - : +- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (46) - : : +- * BroadcastHashJoin Inner BuildRight (45) - : : :- * ColumnarToRow (40) - : : : +- CometFilter (39) - : : : +- CometScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (44) - : : +- * ColumnarToRow (43) - : : +- CometFilter (42) - : : +- CometScan parquet spark_catalog.default.web_sales (41) - : +- ReusedExchange (47) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * ColumnarToRow (74) - +- CometColumnarExchange (73) - +- RowToColumnar (72) - +- * HashAggregate (71) - +- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * ColumnarToRow (61) - : : +- CometFilter (60) - : : +- CometScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (65) - : +- * ColumnarToRow (64) - : +- CometFilter (63) - : +- CometScan parquet spark_catalog.default.web_sales (62) - +- ReusedExchange (68) +TakeOrderedAndProject (71) ++- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (25) + : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- BroadcastExchange (23) + : : : +- * ColumnarToRow (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- ReusedExchange (26) + : +- BroadcastExchange (50) + : +- * Filter (49) + : +- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * ColumnarToRow (36) + : : : +- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (43) + +- BroadcastExchange (68) + +- * HashAggregate (67) + +- Exchange (66) + +- * HashAggregate (65) + +- * Project (64) + +- * BroadcastHashJoin Inner BuildRight (63) + :- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * ColumnarToRow (55) + : : +- CometFilter (54) + : : +- CometScan parquet spark_catalog.default.customer (53) + : +- BroadcastExchange (59) + : +- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (62) (unknown) Scan parquet spark_catalog.default.customer @@ -123,7 +115,7 @@ Join condition: None Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 83] +(10) ReusedExchange [Reuses operator id: 75] Output [2]: [d_date_sk#9, d_year#10] (11) BroadcastHashJoin [codegen id : 3] @@ -143,24 +135,18 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] Aggregate Attributes [1]: [sum#11] Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -(14) RowToColumnar +(14) Exchange Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 16] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] - -(17) HashAggregate [codegen id : 16] +(15) HashAggregate [codegen id : 16] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] -(18) Filter [codegen id : 16] +(16) Filter [codegen id : 16] Input [2]: [customer_id#14, year_total#15] Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) @@ -171,11 +157,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(20) CometFilter +(18) CometFilter Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) -(21) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 6] Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] (unknown) Scan parquet spark_catalog.default.store_sales @@ -186,69 +172,63 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter +(21) CometFilter Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) -(24) ColumnarToRow [codegen id : 4] +(22) ColumnarToRow [codegen id : 4] Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -(25) BroadcastExchange +(23) BroadcastExchange Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [c_customer_sk#16] Right keys [1]: [ss_customer_sk#20] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(25) Project [codegen id : 6] Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -(28) ReusedExchange [Reuses operator id: 87] +(26) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#24, d_year#25] -(29) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#22] Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None -(30) Project [codegen id : 6] +(28) Project [codegen id : 6] Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] -(31) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 6] Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] Aggregate Attributes [1]: [sum#26] Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -(32) RowToColumnar -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] - -(33) CometColumnarExchange -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) ColumnarToRow [codegen id : 7] +(30) Exchange Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] +Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] -(36) BroadcastExchange +(32) BroadcastExchange Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(33) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#28] Join type: Inner @@ -261,11 +241,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(39) CometFilter +(35) CometFilter Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) -(40) ColumnarToRow [codegen id : 10] +(36) ColumnarToRow [codegen id : 10] Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] (unknown) Scan parquet spark_catalog.default.web_sales @@ -276,79 +256,73 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(42) CometFilter +(38) CometFilter Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Condition : isnotnull(ws_bill_customer_sk#36) -(43) ColumnarToRow [codegen id : 8] +(39) ColumnarToRow [codegen id : 8] Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -(44) BroadcastExchange +(40) BroadcastExchange Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 10] +(41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#32] Right keys [1]: [ws_bill_customer_sk#36] Join type: Inner Join condition: None -(46) Project [codegen id : 10] +(42) Project [codegen id : 10] Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -(47) ReusedExchange [Reuses operator id: 83] +(43) ReusedExchange [Reuses operator id: 75] Output [2]: [d_date_sk#40, d_year#41] -(48) BroadcastHashJoin [codegen id : 10] +(44) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#38] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(49) Project [codegen id : 10] +(45) Project [codegen id : 10] Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] -(50) HashAggregate [codegen id : 10] +(46) HashAggregate [codegen id : 10] Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] Aggregate Attributes [1]: [sum#42] Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -(51) RowToColumnar +(47) Exchange Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(52) CometColumnarExchange -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) ColumnarToRow [codegen id : 11] -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] - -(54) HashAggregate [codegen id : 11] +(48) HashAggregate [codegen id : 11] Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] -(55) Filter [codegen id : 11] +(49) Filter [codegen id : 11] Input [2]: [customer_id#45, year_total#46] Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) -(56) BroadcastExchange +(50) BroadcastExchange Input [2]: [customer_id#45, year_total#46] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#45] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(52) Project [codegen id : 16] Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] @@ -359,11 +333,11 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(60) CometFilter +(54) CometFilter Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) -(61) ColumnarToRow [codegen id : 14] +(55) ColumnarToRow [codegen id : 14] Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] (unknown) Scan parquet spark_catalog.default.web_sales @@ -374,89 +348,83 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(63) CometFilter +(57) CometFilter Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Condition : isnotnull(ws_bill_customer_sk#51) -(64) ColumnarToRow [codegen id : 12] +(58) ColumnarToRow [codegen id : 12] Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -(65) BroadcastExchange +(59) BroadcastExchange Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(66) BroadcastHashJoin [codegen id : 14] +(60) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_customer_sk#47] Right keys [1]: [ws_bill_customer_sk#51] Join type: Inner Join condition: None -(67) Project [codegen id : 14] +(61) Project [codegen id : 14] Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -(68) ReusedExchange [Reuses operator id: 87] +(62) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#55, d_year#56] -(69) BroadcastHashJoin [codegen id : 14] +(63) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_sold_date_sk#53] Right keys [1]: [d_date_sk#55] Join type: Inner Join condition: None -(70) Project [codegen id : 14] +(64) Project [codegen id : 14] Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] -(71) HashAggregate [codegen id : 14] +(65) HashAggregate [codegen id : 14] Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] Aggregate Attributes [1]: [sum#57] Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -(72) RowToColumnar -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] - -(73) CometColumnarExchange -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) ColumnarToRow [codegen id : 15] +(66) Exchange Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(75) HashAggregate [codegen id : 15] +(67) HashAggregate [codegen id : 15] Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#52))#44] Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#52))#44,17,2) AS year_total#60] -(76) BroadcastExchange +(68) BroadcastExchange Input [2]: [customer_id#59, year_total#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(69) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#59] Join type: Inner Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) -(78) Project [codegen id : 16] +(70) Project [codegen id : 16] Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] -(79) TakeOrderedAndProject +(71) TakeOrderedAndProject Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (83) -+- * ColumnarToRow (82) - +- CometFilter (81) - +- CometScan parquet spark_catalog.default.date_dim (80) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) (unknown) Scan parquet spark_catalog.default.date_dim @@ -466,22 +434,22 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter +(73) CometFilter Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(82) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(83) BroadcastExchange +(75) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (87) -+- * ColumnarToRow (86) - +- CometFilter (85) - +- CometScan parquet spark_catalog.default.date_dim (84) +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) (unknown) Scan parquet spark_catalog.default.date_dim @@ -491,19 +459,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(85) CometFilter +(77) CometFilter Input [2]: [d_date_sk#24, d_year#25] Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) -(86) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] -(87) BroadcastExchange +(79) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index aa010c7b45..26989b0c00 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -7,11 +7,41 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - RowToColumnar - WholeStageCodegen (3) + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 + WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -22,109 +52,71 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) + BroadcastExchange #6 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 WholeStageCodegen (11) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - RowToColumnar - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 98da31a346..1c54164074 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -1,138 +1,133 @@ == Physical Plan == -TakeOrderedAndProject (134) -+- * Project (133) - +- * SortMergeJoin Inner (132) - :- * ColumnarToRow (72) - : +- CometSort (71) - : +- CometColumnarExchange (70) - : +- CometFilter (69) - : +- CometHashAggregate (68) - : +- CometColumnarExchange (67) - : +- CometHashAggregate (66) - : +- CometHashAggregate (65) - : +- CometColumnarExchange (64) - : +- CometHashAggregate (63) - : +- CometUnion (62) - : :- CometProject (23) - : : +- CometSortMergeJoin (22) - : : :- CometSort (16) - : : : +- CometColumnarExchange (15) - : : : +- RowToColumnar (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- CometSort (21) - : : +- CometColumnarExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan parquet spark_catalog.default.catalog_returns (17) - : :- CometProject (42) - : : +- CometSortMergeJoin (41) - : : :- CometSort (35) - : : : +- CometColumnarExchange (34) - : : : +- RowToColumnar (33) - : : : +- * Project (32) - : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : :- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * ColumnarToRow (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) - : : : : +- ReusedExchange (27) - : : : +- ReusedExchange (30) - : : +- CometSort (40) - : : +- CometColumnarExchange (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.store_returns (36) - : +- CometProject (61) - : +- CometSortMergeJoin (60) - : :- CometSort (54) - : : +- CometColumnarExchange (53) - : : +- RowToColumnar (52) - : : +- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * ColumnarToRow (45) - : : : : +- CometFilter (44) - : : : : +- CometScan parquet spark_catalog.default.web_sales (43) - : : : +- ReusedExchange (46) - : : +- ReusedExchange (49) - : +- CometSort (59) - : +- CometColumnarExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan parquet spark_catalog.default.web_returns (55) - +- * ColumnarToRow (131) - +- CometSort (130) - +- CometColumnarExchange (129) - +- CometFilter (128) - +- CometHashAggregate (127) - +- CometColumnarExchange (126) - +- CometHashAggregate (125) - +- CometHashAggregate (124) - +- CometColumnarExchange (123) - +- CometHashAggregate (122) - +- CometUnion (121) - :- CometProject (88) - : +- CometSortMergeJoin (87) - : :- CometSort (84) - : : +- CometColumnarExchange (83) - : : +- RowToColumnar (82) - : : +- * Project (81) - : : +- * BroadcastHashJoin Inner BuildRight (80) - : : :- * Project (78) - : : : +- * BroadcastHashJoin Inner BuildRight (77) - : : : :- * ColumnarToRow (75) - : : : : +- CometFilter (74) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) - : : : +- ReusedExchange (76) - : : +- ReusedExchange (79) - : +- CometSort (86) - : +- ReusedExchange (85) - :- CometProject (104) - : +- CometSortMergeJoin (103) - : :- CometSort (100) - : : +- CometColumnarExchange (99) - : : +- RowToColumnar (98) - : : +- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * Project (94) - : : : +- * BroadcastHashJoin Inner BuildRight (93) - : : : :- * ColumnarToRow (91) - : : : : +- CometFilter (90) - : : : : +- CometScan parquet spark_catalog.default.store_sales (89) - : : : +- ReusedExchange (92) - : : +- ReusedExchange (95) - : +- CometSort (102) - : +- ReusedExchange (101) - +- CometProject (120) - +- CometSortMergeJoin (119) - :- CometSort (116) - : +- CometColumnarExchange (115) - : +- RowToColumnar (114) - : +- * Project (113) - : +- * BroadcastHashJoin Inner BuildRight (112) - : :- * Project (110) - : : +- * BroadcastHashJoin Inner BuildRight (109) - : : :- * ColumnarToRow (107) - : : : +- CometFilter (106) - : : : +- CometScan parquet spark_catalog.default.web_sales (105) - : : +- ReusedExchange (108) - : +- ReusedExchange (111) - +- CometSort (118) - +- ReusedExchange (117) +TakeOrderedAndProject (129) ++- * Project (128) + +- * SortMergeJoin Inner (127) + :- * Sort (71) + : +- Exchange (70) + : +- * Filter (69) + : +- * HashAggregate (68) + : +- Exchange (67) + : +- * HashAggregate (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- Union (62) + : :- * Project (23) + : : +- * SortMergeJoin LeftOuter (22) + : : :- * Sort (15) + : : : +- Exchange (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- ReusedExchange (11) + : : +- * Sort (21) + : : +- Exchange (20) + : : +- * ColumnarToRow (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- * Project (42) + : : +- * SortMergeJoin LeftOuter (41) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : :- * Project (29) + : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : :- * ColumnarToRow (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- * Sort (40) + : : +- Exchange (39) + : : +- * ColumnarToRow (38) + : : +- CometProject (37) + : : +- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.store_returns (35) + : +- * Project (61) + : +- * SortMergeJoin LeftOuter (60) + : :- * Sort (53) + : : +- Exchange (52) + : : +- * Project (51) + : : +- * BroadcastHashJoin Inner BuildRight (50) + : : :- * Project (48) + : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : :- * ColumnarToRow (45) + : : : : +- CometFilter (44) + : : : : +- CometScan parquet spark_catalog.default.web_sales (43) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- * Sort (59) + : +- Exchange (58) + : +- * ColumnarToRow (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.web_returns (54) + +- * Sort (126) + +- Exchange (125) + +- * Filter (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- * HashAggregate (120) + +- Exchange (119) + +- * HashAggregate (118) + +- Union (117) + :- * Project (86) + : +- * SortMergeJoin LeftOuter (85) + : :- * Sort (82) + : : +- Exchange (81) + : : +- * Project (80) + : : +- * BroadcastHashJoin Inner BuildRight (79) + : : :- * Project (77) + : : : +- * BroadcastHashJoin Inner BuildRight (76) + : : : :- * ColumnarToRow (74) + : : : : +- CometFilter (73) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) + : : : +- ReusedExchange (75) + : : +- ReusedExchange (78) + : +- * Sort (84) + : +- ReusedExchange (83) + :- * Project (101) + : +- * SortMergeJoin LeftOuter (100) + : :- * Sort (97) + : : +- Exchange (96) + : : +- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * Project (92) + : : : +- * BroadcastHashJoin Inner BuildRight (91) + : : : :- * ColumnarToRow (89) + : : : : +- CometFilter (88) + : : : : +- CometScan parquet spark_catalog.default.store_sales (87) + : : : +- ReusedExchange (90) + : : +- ReusedExchange (93) + : +- * Sort (99) + : +- ReusedExchange (98) + +- * Project (116) + +- * SortMergeJoin LeftOuter (115) + :- * Sort (112) + : +- Exchange (111) + : +- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * Project (107) + : : +- * BroadcastHashJoin Inner BuildRight (106) + : : :- * ColumnarToRow (104) + : : : +- CometFilter (103) + : : : +- CometScan parquet spark_catalog.default.web_sales (102) + : : +- ReusedExchange (105) + : +- ReusedExchange (108) + +- * Sort (114) + +- ReusedExchange (113) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -182,7 +177,7 @@ Join condition: None Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 138] +(11) ReusedExchange [Reuses operator id: 133] Output [2]: [d_date_sk#13, d_year#14] (12) BroadcastHashJoin [codegen id : 3] @@ -195,16 +190,13 @@ Join condition: None Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -(14) RowToColumnar +(14) Exchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange +(15) Sort [codegen id : 4] Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] @@ -213,30 +205,34 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(18) CometFilter +(17) CometFilter Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) -(19) CometProject +(18) CometProject Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(20) CometColumnarExchange +(19) ColumnarToRow [codegen id : 5] +Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] + +(20) Exchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometSort +(21) Sort [codegen id : 6] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] +Arguments: [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST], false, 0 -(22) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter +(22) SortMergeJoin [codegen id : 7] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#16, cr_item_sk#15] +Join type: LeftOuter +Join condition: None -(23) CometProject +(23) Project [codegen id : 7] +Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] (unknown) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] @@ -250,45 +246,42 @@ ReadSchema: struct -(37) CometFilter +(36) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(38) CometProject +(37) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(39) CometColumnarExchange +(38) ColumnarToRow [codegen id : 12] +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] + +(39) Exchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(40) CometSort +(40) Sort [codegen id : 13] Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] +Arguments: [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST], false, 0 -(41) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter +(41) SortMergeJoin [codegen id : 14] +Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] +Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] +Join type: LeftOuter +Join condition: None -(42) CometProject +(42) Project [codegen id : 14] +Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] (unknown) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] @@ -334,45 +331,42 @@ ReadSchema: struct -(56) CometFilter +(55) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(57) CometProject +(56) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(58) CometColumnarExchange +(57) ColumnarToRow [codegen id : 19] Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(59) CometSort +(58) Exchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(60) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter +(59) Sort [codegen id : 20] +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST], false, 0 + +(60) SortMergeJoin [codegen id : 21] +Left keys [2]: [ws_order_number#43, ws_item_sk#42] +Right keys [2]: [wr_order_number#56, wr_item_sk#55] +Join type: LeftOuter +Join condition: None -(61) CometProject +(61) Project [codegen id : 21] +Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] -(62) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] +(62) Union -(63) CometHashAggregate +(63) HashAggregate [codegen id : 22] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(64) CometColumnarExchange +(64) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(65) CometHashAggregate +(65) HashAggregate [codegen id : 23] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(66) CometHashAggregate +(66) HashAggregate [codegen id : 23] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum#62, sum#63] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -(67) CometColumnarExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(67) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(68) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] +(68) HashAggregate [codegen id : 24] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] -(69) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) +(69) Filter [codegen id : 24] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Condition : isnotnull(sales_cnt#68) -(70) CometColumnarExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(70) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(71) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(72) ColumnarToRow [codegen id : 10] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] +(71) Sort [codegen id : 25] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(74) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) +(73) CometFilter +Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Condition : isnotnull(cs_item_sk#70) -(75) ColumnarToRow [codegen id : 13] -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +(74) ColumnarToRow [codegen id : 28] +Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] -(76) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +(75) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(77) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_item_sk#66] -Right keys [1]: [i_item_sk#72] +(76) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [cs_item_sk#70] +Right keys [1]: [i_item_sk#76] Join type: Inner Join condition: None -(78) Project [codegen id : 13] -Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +(77) Project [codegen id : 28] +Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(79) ReusedExchange [Reuses operator id: 142] -Output [2]: [d_date_sk#77, d_year#78] +(78) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#81, d_year#82] -(80) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_sold_date_sk#70] -Right keys [1]: [d_date_sk#77] +(79) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [cs_sold_date_sk#74] +Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(81) Project [codegen id : 13] -Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] - -(82) RowToColumnar -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +(80) Project [codegen id : 28] +Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] -(83) CometColumnarExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(81) Exchange +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(84) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] +(82) Sort [codegen id : 29] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 -(85) ReusedExchange [Reuses operator id: 20] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] +(83) ReusedExchange [Reuses operator id: 20] +Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(86) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] +(84) Sort [codegen id : 31] +Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +Arguments: [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST], false, 0 -(87) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter +(85) SortMergeJoin [codegen id : 32] +Left keys [2]: [cs_order_number#71, cs_item_sk#70] +Right keys [2]: [cr_order_number#84, cr_item_sk#83] +Join type: LeftOuter +Join condition: None -(88) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] +(86) Project [codegen id : 32] +Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#20, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#21] +Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] (unknown) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] +PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(90) CometFilter -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) +(88) CometFilter +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Condition : isnotnull(ss_item_sk#87) -(91) ColumnarToRow [codegen id : 16] -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +(89) ColumnarToRow [codegen id : 35] +Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -(92) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +(90) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(93) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ss_item_sk#83] -Right keys [1]: [i_item_sk#89] +(91) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ss_item_sk#87] +Right keys [1]: [i_item_sk#93] Join type: Inner Join condition: None -(94) Project [codegen id : 16] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +(92) Project [codegen id : 35] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(95) ReusedExchange [Reuses operator id: 142] -Output [2]: [d_date_sk#94, d_year#95] +(93) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#98, d_year#99] -(96) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ss_sold_date_sk#87] -Right keys [1]: [d_date_sk#94] +(94) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ss_sold_date_sk#91] +Right keys [1]: [d_date_sk#98] Join type: Inner Join condition: None -(97) Project [codegen id : 16] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] +(95) Project [codegen id : 35] +Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] -(98) RowToColumnar -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +(96) Exchange +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(99) CometColumnarExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +(97) Sort [codegen id : 36] +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 -(100) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] +(98) ReusedExchange [Reuses operator id: 39] +Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -(101) ReusedExchange [Reuses operator id: 39] -Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] +(99) Sort [codegen id : 38] +Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +Arguments: [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST], false, 0 -(102) CometSort -Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] - -(103) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter +(100) SortMergeJoin [codegen id : 39] +Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] +Right keys [2]: [sr_ticket_number#101, sr_item_sk#100] +Join type: LeftOuter +Join condition: None -(104) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] +(101) Project [codegen id : 39] +Output [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#40, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#41] +Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] (unknown) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] +PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(106) CometFilter -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Condition : isnotnull(ws_item_sk#100) +(103) CometFilter +Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Condition : isnotnull(ws_item_sk#104) -(107) ColumnarToRow [codegen id : 19] -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +(104) ColumnarToRow [codegen id : 42] +Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -(108) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +(105) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(109) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_item_sk#100] -Right keys [1]: [i_item_sk#106] +(106) BroadcastHashJoin [codegen id : 42] +Left keys [1]: [ws_item_sk#104] +Right keys [1]: [i_item_sk#110] Join type: Inner Join condition: None -(110) Project [codegen id : 19] -Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +(107) Project [codegen id : 42] +Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(111) ReusedExchange [Reuses operator id: 142] -Output [2]: [d_date_sk#111, d_year#112] +(108) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#115, d_year#116] -(112) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#104] -Right keys [1]: [d_date_sk#111] +(109) BroadcastHashJoin [codegen id : 42] +Left keys [1]: [ws_sold_date_sk#108] +Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None -(113) Project [codegen id : 19] -Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] - -(114) RowToColumnar -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +(110) Project [codegen id : 42] +Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] -(115) CometColumnarExchange -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +(111) Exchange +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(116) CometSort -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] +(112) Sort [codegen id : 43] +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST], false, 0 -(117) ReusedExchange [Reuses operator id: 58] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] +(113) ReusedExchange [Reuses operator id: 58] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(118) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] +(114) Sort [codegen id : 45] +Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST], false, 0 -(119) CometSortMergeJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter +(115) SortMergeJoin [codegen id : 46] +Left keys [2]: [ws_order_number#105, ws_item_sk#104] +Right keys [2]: [wr_order_number#118, wr_item_sk#117] +Join type: LeftOuter +Join condition: None -(120) CometProject -Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] +(116) Project [codegen id : 46] +Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#60, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#61] +Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(121) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] +(117) Union -(122) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +(118) HashAggregate [codegen id : 47] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(123) CometColumnarExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(119) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(124) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] +(120) HashAggregate [codegen id : 48] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(125) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +(121) HashAggregate [codegen id : 48] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] +Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum#62, sum#121] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -(126) CometColumnarExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(122) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(127) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +(123) HashAggregate [codegen id : 49] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] +Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#20)#66 AS sales_cnt#123, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#124] -(128) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Condition : isnotnull(sales_cnt#118) - -(129) CometColumnarExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] +(124) Filter [codegen id : 49] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] +Condition : isnotnull(sales_cnt#123) -(130) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] +(125) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] +Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(131) ColumnarToRow [codegen id : 20] -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +(126) Sort [codegen id : 50] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] +Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 -(132) SortMergeJoin [codegen id : 21] +(127) SortMergeJoin [codegen id : 51] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right keys [4]: [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Join type: Inner -Join condition: ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) -(133) Project [codegen id : 21] -Output [10]: [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] +(128) Project [codegen id : 51] +Output [10]: [d_year#82 AS prev_year#125, d_year#14 AS year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#123 AS prev_yr_cnt#127, sales_cnt#68 AS curr_yr_cnt#128, (sales_cnt#68 - sales_cnt#123) AS sales_cnt_diff#129, (sales_amt#69 - sales_amt#124) AS sales_amt_diff#130] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -(134) TakeOrderedAndProject -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -Arguments: 100, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] +(129) TakeOrderedAndProject +Input [10]: [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] +Arguments: 100, [sales_cnt_diff#129 ASC NULLS FIRST, sales_amt_diff#130 ASC NULLS FIRST], [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (138) -+- * ColumnarToRow (137) - +- CometFilter (136) - +- CometScan parquet spark_catalog.default.date_dim (135) +BroadcastExchange (133) ++- * ColumnarToRow (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) (unknown) Scan parquet spark_catalog.default.date_dim @@ -736,14 +732,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(136) CometFilter +(131) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(137) ColumnarToRow [codegen id : 1] +(132) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(138) BroadcastExchange +(133) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] @@ -751,33 +747,33 @@ Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (142) -+- * ColumnarToRow (141) - +- CometFilter (140) - +- CometScan parquet spark_catalog.default.date_dim (139) +Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (137) ++- * ColumnarToRow (136) + +- CometFilter (135) + +- CometScan parquet spark_catalog.default.date_dim (134) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] +Output [2]: [d_date_sk#81, d_year#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(140) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) +(135) CometFilter +Input [2]: [d_date_sk#81, d_year#82] +Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(141) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] +(136) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#81, d_year#82] -(142) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] +(137) BroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 89 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 +Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 +Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt index a0e22e72fb..9939b2fe21 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt @@ -1,185 +1,240 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt] - WholeStageCodegen (21) + WholeStageCodegen (51) Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter - WholeStageCodegen (10) - ColumnarToRow + WholeStageCodegen (25) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - CometSort [i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [sales_cnt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - RowToColumnar - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + WholeStageCodegen (24) + Filter [sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + WholeStageCodegen (23) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + WholeStageCodegen (22) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (7) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (3) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + WholeStageCodegen (6) + Sort [cr_order_number,cr_item_sk] InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow + Exchange [cr_order_number,cr_item_sk] #7 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + WholeStageCodegen (14) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (11) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (10) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_order_number,cr_item_sk] - CometColumnarExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - RowToColumnar - WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow + WholeStageCodegen (13) + Sort [sr_ticket_number,sr_item_sk] InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Exchange [sr_ticket_number,sr_item_sk] #9 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + WholeStageCodegen (21) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - RowToColumnar - WholeStageCodegen (9) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow + WholeStageCodegen (18) + Sort [ws_order_number,ws_item_sk] InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Exchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (17) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_order_number,wr_item_sk] - CometColumnarExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + WholeStageCodegen (20) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #11 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter - WholeStageCodegen (20) - ColumnarToRow + WholeStageCodegen (50) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - CometSort [i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [sales_cnt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - RowToColumnar - WholeStageCodegen (13) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + WholeStageCodegen (49) + Filter [sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + WholeStageCodegen (48) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 + WholeStageCodegen (47) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (32) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (29) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (28) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + InputAdapter + WholeStageCodegen (31) + Sort [cr_order_number,cr_item_sk] InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - ColumnarToRow + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + WholeStageCodegen (39) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (36) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (35) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_order_number,cr_item_sk] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - RowToColumnar - WholeStageCodegen (16) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow + WholeStageCodegen (38) + Sort [sr_ticket_number,sr_item_sk] InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + WholeStageCodegen (46) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - RowToColumnar - WholeStageCodegen (19) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow + WholeStageCodegen (43) + Sort [ws_order_number,ws_item_sk] InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Exchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (42) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_order_number,wr_item_sk] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + WholeStageCodegen (45) + Sort [wr_order_number,wr_item_sk] + InputAdapter + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt index b2b0068947..82c7d9b244 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt @@ -1,124 +1,102 @@ == Physical Plan == -* ColumnarToRow (120) -+- CometTakeOrderedAndProject (119) - +- CometHashAggregate (118) - +- CometColumnarExchange (117) - +- RowToColumnar (116) - +- * HashAggregate (115) - +- Union (114) - :- * HashAggregate (97) - : +- * ColumnarToRow (96) - : +- CometColumnarExchange (95) - : +- RowToColumnar (94) - : +- * HashAggregate (93) - : +- Union (92) - : :- * Project (34) - : : +- * BroadcastHashJoin LeftOuter BuildRight (33) - : : :- * HashAggregate (17) - : : : +- * ColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- RowToColumnar (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.store (7) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- * ColumnarToRow (30) - : : +- CometColumnarExchange (29) - : : +- RowToColumnar (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * ColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometScan parquet spark_catalog.default.store_returns (18) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : :- * Project (57) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (56) - : : :- BroadcastExchange (45) - : : : +- * HashAggregate (44) - : : : +- * ColumnarToRow (43) - : : : +- CometColumnarExchange (42) - : : : +- RowToColumnar (41) - : : : +- * HashAggregate (40) - : : : +- * Project (39) - : : : +- * BroadcastHashJoin Inner BuildRight (38) - : : : :- * ColumnarToRow (36) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (35) - : : : +- ReusedExchange (37) - : : +- * HashAggregate (55) - : : +- * ColumnarToRow (54) - : : +- CometColumnarExchange (53) - : : +- RowToColumnar (52) - : : +- * HashAggregate (51) - : : +- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * ColumnarToRow (47) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (46) - : : +- ReusedExchange (48) - : +- * Project (91) - : +- * BroadcastHashJoin LeftOuter BuildRight (90) - : :- * HashAggregate (74) - : : +- * ColumnarToRow (73) - : : +- CometColumnarExchange (72) - : : +- RowToColumnar (71) - : : +- * HashAggregate (70) - : : +- * Project (69) - : : +- * BroadcastHashJoin Inner BuildRight (68) - : : :- * Project (63) - : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : :- * ColumnarToRow (60) - : : : : +- CometFilter (59) - : : : : +- CometScan parquet spark_catalog.default.web_sales (58) - : : : +- ReusedExchange (61) - : : +- BroadcastExchange (67) - : : +- * ColumnarToRow (66) - : : +- CometFilter (65) - : : +- CometScan parquet spark_catalog.default.web_page (64) - : +- BroadcastExchange (89) - : +- * HashAggregate (88) - : +- * ColumnarToRow (87) - : +- CometColumnarExchange (86) - : +- RowToColumnar (85) - : +- * HashAggregate (84) - : +- * Project (83) - : +- * BroadcastHashJoin Inner BuildRight (82) - : :- * Project (80) - : : +- * BroadcastHashJoin Inner BuildRight (79) - : : :- * ColumnarToRow (77) - : : : +- CometFilter (76) - : : : +- CometScan parquet spark_catalog.default.web_returns (75) - : : +- ReusedExchange (78) - : +- ReusedExchange (81) - :- * HashAggregate (105) - : +- * ColumnarToRow (104) - : +- CometColumnarExchange (103) - : +- RowToColumnar (102) - : +- * HashAggregate (101) - : +- * HashAggregate (100) - : +- * ColumnarToRow (99) - : +- ReusedExchange (98) - +- * HashAggregate (113) - +- * ColumnarToRow (112) - +- CometColumnarExchange (111) - +- RowToColumnar (110) - +- * HashAggregate (109) - +- * HashAggregate (108) - +- * ColumnarToRow (107) - +- ReusedExchange (106) +TakeOrderedAndProject (98) ++- * HashAggregate (97) + +- Exchange (96) + +- * HashAggregate (95) + +- Union (94) + :- * HashAggregate (83) + : +- Exchange (82) + : +- * HashAggregate (81) + : +- Union (80) + : :- * Project (30) + : : +- * BroadcastHashJoin LeftOuter BuildRight (29) + : : :- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan parquet spark_catalog.default.store (7) + : : +- BroadcastExchange (28) + : : +- * HashAggregate (27) + : : +- Exchange (26) + : : +- * HashAggregate (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * ColumnarToRow (18) + : : : : +- CometFilter (17) + : : : : +- CometScan parquet spark_catalog.default.store_returns (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : :- * Project (49) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (48) + : : :- BroadcastExchange (39) + : : : +- * HashAggregate (38) + : : : +- Exchange (37) + : : : +- * HashAggregate (36) + : : : +- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (32) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : : +- ReusedExchange (33) + : : +- * HashAggregate (47) + : : +- Exchange (46) + : : +- * HashAggregate (45) + : : +- * Project (44) + : : +- * BroadcastHashJoin Inner BuildRight (43) + : : :- * ColumnarToRow (41) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (40) + : : +- ReusedExchange (42) + : +- * Project (79) + : +- * BroadcastHashJoin LeftOuter BuildRight (78) + : :- * HashAggregate (64) + : : +- Exchange (63) + : : +- * HashAggregate (62) + : : +- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * ColumnarToRow (52) + : : : : +- CometFilter (51) + : : : : +- CometScan parquet spark_catalog.default.web_sales (50) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (59) + : : +- * ColumnarToRow (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.web_page (56) + : +- BroadcastExchange (77) + : +- * HashAggregate (76) + : +- Exchange (75) + : +- * HashAggregate (74) + : +- * Project (73) + : +- * BroadcastHashJoin Inner BuildRight (72) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * ColumnarToRow (67) + : : : +- CometFilter (66) + : : : +- CometScan parquet spark_catalog.default.web_returns (65) + : : +- ReusedExchange (68) + : +- ReusedExchange (71) + :- * HashAggregate (88) + : +- Exchange (87) + : +- * HashAggregate (86) + : +- * HashAggregate (85) + : +- ReusedExchange (84) + +- * HashAggregate (93) + +- Exchange (92) + +- * HashAggregate (91) + +- * HashAggregate (90) + +- ReusedExchange (89) (unknown) Scan parquet spark_catalog.default.store_sales @@ -136,7 +114,7 @@ Condition : isnotnull(ss_store_sk#1) (3) ColumnarToRow [codegen id : 3] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(4) ReusedExchange [Reuses operator id: 125] +(4) ReusedExchange [Reuses operator id: 103] Output [1]: [d_date_sk#6] (5) BroadcastHashJoin [codegen id : 3] @@ -184,17 +162,11 @@ Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(Un Aggregate Attributes [2]: [sum#8, sum#9] Results [3]: [s_store_sk#7, sum#10, sum#11] -(14) RowToColumnar +(14) Exchange Input [3]: [s_store_sk#7, sum#10, sum#11] +Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] - -(17) HashAggregate [codegen id : 8] +(15) HashAggregate [codegen id : 8] Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] @@ -209,74 +181,68 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Condition : isnotnull(sr_store_sk#16) -(20) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 6] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -(21) ReusedExchange [Reuses operator id: 125] +(19) ReusedExchange [Reuses operator id: 103] Output [1]: [d_date_sk#21] -(22) BroadcastHashJoin [codegen id : 6] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#19] Right keys [1]: [d_date_sk#21] Join type: Inner Join condition: None -(23) Project [codegen id : 6] +(21) Project [codegen id : 6] Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] -(24) ReusedExchange [Reuses operator id: 10] +(22) ReusedExchange [Reuses operator id: 10] Output [1]: [s_store_sk#22] -(25) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#16] Right keys [1]: [s_store_sk#22] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -(27) HashAggregate [codegen id : 6] +(25) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] Keys [1]: [s_store_sk#22] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] Aggregate Attributes [2]: [sum#23, sum#24] Results [3]: [s_store_sk#22, sum#25, sum#26] -(28) RowToColumnar -Input [3]: [s_store_sk#22, sum#25, sum#26] - -(29) CometColumnarExchange +(26) Exchange Input [3]: [s_store_sk#22, sum#25, sum#26] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(30) ColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#22, sum#25, sum#26] - -(31) HashAggregate [codegen id : 7] +(27) HashAggregate [codegen id : 7] Input [3]: [s_store_sk#22, sum#25, sum#26] Keys [1]: [s_store_sk#22] Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] -(32) BroadcastExchange +(28) BroadcastExchange Input [3]: [s_store_sk#22, returns#29, profit_loss#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 8] +(29) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#7] Right keys [1]: [s_store_sk#22] Join type: LeftOuter Join condition: None -(34) Project [codegen id : 8] +(30) Project [codegen id : 8] Output [5]: [store channel AS channel#31, s_store_sk#7 AS id#32, sales#14, coalesce(returns#29, 0.00) AS returns#33, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#34] Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] @@ -287,47 +253,41 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 10] +(32) ColumnarToRow [codegen id : 10] Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] -(37) ReusedExchange [Reuses operator id: 125] +(33) ReusedExchange [Reuses operator id: 103] Output [1]: [d_date_sk#40] -(38) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#38] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(39) Project [codegen id : 10] +(35) Project [codegen id : 10] Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] -(40) HashAggregate [codegen id : 10] +(36) HashAggregate [codegen id : 10] Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] Keys [1]: [cs_call_center_sk#35] Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] Aggregate Attributes [2]: [sum#41, sum#42] Results [3]: [cs_call_center_sk#35, sum#43, sum#44] -(41) RowToColumnar -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(42) CometColumnarExchange -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) ColumnarToRow [codegen id : 11] +(37) Exchange Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(44) HashAggregate [codegen id : 11] +(38) HashAggregate [codegen id : 11] Input [3]: [cs_call_center_sk#35, sum#43, sum#44] Keys [1]: [cs_call_center_sk#35] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] -(45) BroadcastExchange +(39) BroadcastExchange Input [3]: [cs_call_center_sk#35, sales#47, profit#48] Arguments: IdentityBroadcastMode, [plan_id=6] @@ -338,51 +298,45 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 13] +(41) ColumnarToRow [codegen id : 13] Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -(48) ReusedExchange [Reuses operator id: 125] +(42) ReusedExchange [Reuses operator id: 103] Output [1]: [d_date_sk#53] -(49) BroadcastHashJoin [codegen id : 13] +(43) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cr_returned_date_sk#51] Right keys [1]: [d_date_sk#53] Join type: Inner Join condition: None -(50) Project [codegen id : 13] +(44) Project [codegen id : 13] Output [2]: [cr_return_amount#49, cr_net_loss#50] Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] -(51) HashAggregate [codegen id : 13] +(45) HashAggregate [codegen id : 13] Input [2]: [cr_return_amount#49, cr_net_loss#50] Keys: [] Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] Aggregate Attributes [2]: [sum#54, sum#55] Results [2]: [sum#56, sum#57] -(52) RowToColumnar +(46) Exchange Input [2]: [sum#56, sum#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(53) CometColumnarExchange -Input [2]: [sum#56, sum#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(54) ColumnarToRow -Input [2]: [sum#56, sum#57] - -(55) HashAggregate +(47) HashAggregate Input [2]: [sum#56, sum#57] Keys: [] Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] -(56) BroadcastNestedLoopJoin [codegen id : 14] +(48) BroadcastNestedLoopJoin [codegen id : 14] Join type: Inner Join condition: None -(57) Project [codegen id : 14] +(49) Project [codegen id : 14] Output [5]: [catalog channel AS channel#62, cs_call_center_sk#35 AS id#63, sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#64] Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] @@ -394,23 +348,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(59) CometFilter +(51) CometFilter Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] Condition : isnotnull(ws_web_page_sk#65) -(60) ColumnarToRow [codegen id : 17] +(52) ColumnarToRow [codegen id : 17] Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -(61) ReusedExchange [Reuses operator id: 125] +(53) ReusedExchange [Reuses operator id: 103] Output [1]: [d_date_sk#70] -(62) BroadcastHashJoin [codegen id : 17] +(54) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_sold_date_sk#68] Right keys [1]: [d_date_sk#70] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(55) Project [codegen id : 17] Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] @@ -421,45 +375,39 @@ Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(65) CometFilter +(57) CometFilter Input [1]: [wp_web_page_sk#71] Condition : isnotnull(wp_web_page_sk#71) -(66) ColumnarToRow [codegen id : 16] +(58) ColumnarToRow [codegen id : 16] Input [1]: [wp_web_page_sk#71] -(67) BroadcastExchange +(59) BroadcastExchange Input [1]: [wp_web_page_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(68) BroadcastHashJoin [codegen id : 17] +(60) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_web_page_sk#65] Right keys [1]: [wp_web_page_sk#71] Join type: Inner Join condition: None -(69) Project [codegen id : 17] +(61) Project [codegen id : 17] Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -(70) HashAggregate [codegen id : 17] +(62) HashAggregate [codegen id : 17] Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] Keys [1]: [wp_web_page_sk#71] Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] Aggregate Attributes [2]: [sum#72, sum#73] Results [3]: [wp_web_page_sk#71, sum#74, sum#75] -(71) RowToColumnar -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(72) CometColumnarExchange +(63) Exchange Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(73) ColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(74) HashAggregate [codegen id : 22] +(64) HashAggregate [codegen id : 22] Input [3]: [wp_web_page_sk#71, sum#74, sum#75] Keys [1]: [wp_web_page_sk#71] Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] @@ -474,213 +422,179 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(76) CometFilter +(66) CometFilter Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] Condition : isnotnull(wr_web_page_sk#80) -(77) ColumnarToRow [codegen id : 20] +(67) ColumnarToRow [codegen id : 20] Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] -(78) ReusedExchange [Reuses operator id: 125] +(68) ReusedExchange [Reuses operator id: 103] Output [1]: [d_date_sk#85] -(79) BroadcastHashJoin [codegen id : 20] +(69) BroadcastHashJoin [codegen id : 20] Left keys [1]: [wr_returned_date_sk#83] Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None -(80) Project [codegen id : 20] +(70) Project [codegen id : 20] Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] -(81) ReusedExchange [Reuses operator id: 67] +(71) ReusedExchange [Reuses operator id: 59] Output [1]: [wp_web_page_sk#86] -(82) BroadcastHashJoin [codegen id : 20] +(72) BroadcastHashJoin [codegen id : 20] Left keys [1]: [wr_web_page_sk#80] Right keys [1]: [wp_web_page_sk#86] Join type: Inner Join condition: None -(83) Project [codegen id : 20] +(73) Project [codegen id : 20] Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -(84) HashAggregate [codegen id : 20] +(74) HashAggregate [codegen id : 20] Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] Keys [1]: [wp_web_page_sk#86] Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] Aggregate Attributes [2]: [sum#87, sum#88] Results [3]: [wp_web_page_sk#86, sum#89, sum#90] -(85) RowToColumnar -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(86) CometColumnarExchange -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(87) ColumnarToRow [codegen id : 21] +(75) Exchange Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(88) HashAggregate [codegen id : 21] +(76) HashAggregate [codegen id : 21] Input [3]: [wp_web_page_sk#86, sum#89, sum#90] Keys [1]: [wp_web_page_sk#86] Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] -(89) BroadcastExchange +(77) BroadcastExchange Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(90) BroadcastHashJoin [codegen id : 22] +(78) BroadcastHashJoin [codegen id : 22] Left keys [1]: [wp_web_page_sk#71] Right keys [1]: [wp_web_page_sk#86] Join type: LeftOuter Join condition: None -(91) Project [codegen id : 22] +(79) Project [codegen id : 22] Output [5]: [web channel AS channel#95, wp_web_page_sk#71 AS id#96, sales#78, coalesce(returns#93, 0.00) AS returns#97, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#98] Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] -(92) Union +(80) Union -(93) HashAggregate [codegen id : 23] +(81) HashAggregate [codegen id : 23] Input [5]: [channel#31, id#32, sales#14, returns#33, profit#34] Keys [2]: [channel#31, id#32] Functions [3]: [partial_sum(sales#14), partial_sum(returns#33), partial_sum(profit#34)] Aggregate Attributes [6]: [sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] Results [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -(94) RowToColumnar -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(95) CometColumnarExchange +(82) Exchange Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(96) ColumnarToRow [codegen id : 24] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(97) HashAggregate [codegen id : 24] +(83) HashAggregate [codegen id : 24] Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] Keys [2]: [channel#31, id#32] Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] Results [5]: [channel#31, id#32, cast(sum(sales#14)#111 as decimal(37,2)) AS sales#114, cast(sum(returns#33)#112 as decimal(37,2)) AS returns#115, cast(sum(profit#34)#113 as decimal(38,2)) AS profit#116] -(98) ReusedExchange [Reuses operator id: 95] +(84) ReusedExchange [Reuses operator id: 82] Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -(99) ColumnarToRow [codegen id : 48] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(100) HashAggregate [codegen id : 48] +(85) HashAggregate [codegen id : 48] Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] Keys [2]: [channel#31, id#32] Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] Results [4]: [channel#31, sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] -(101) HashAggregate [codegen id : 48] +(86) HashAggregate [codegen id : 48] Input [4]: [channel#31, sales#117, returns#118, profit#119] Keys [1]: [channel#31] Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] Aggregate Attributes [6]: [sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] Results [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -(102) RowToColumnar -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(103) CometColumnarExchange +(87) Exchange Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(104) ColumnarToRow [codegen id : 49] -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(105) HashAggregate [codegen id : 49] +(88) HashAggregate [codegen id : 49] Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] Keys [1]: [channel#31] Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] Aggregate Attributes [3]: [sum(sales#117)#132, sum(returns#118)#133, sum(profit#119)#134] Results [5]: [channel#31, null AS id#135, sum(sales#117)#132 AS sales#136, sum(returns#118)#133 AS returns#137, sum(profit#119)#134 AS profit#138] -(106) ReusedExchange [Reuses operator id: 95] +(89) ReusedExchange [Reuses operator id: 82] Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -(107) ColumnarToRow [codegen id : 73] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(108) HashAggregate [codegen id : 73] +(90) HashAggregate [codegen id : 73] Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] Keys [2]: [channel#31, id#32] Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] Results [3]: [sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] -(109) HashAggregate [codegen id : 73] +(91) HashAggregate [codegen id : 73] Input [3]: [sales#117, returns#118, profit#119] Keys: [] Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] Aggregate Attributes [6]: [sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Results [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] -(110) RowToColumnar -Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] - -(111) CometColumnarExchange +(92) Exchange Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(112) ColumnarToRow [codegen id : 74] -Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] - -(113) HashAggregate [codegen id : 74] +(93) HashAggregate [codegen id : 74] Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] Keys: [] Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] Aggregate Attributes [3]: [sum(sales#117)#151, sum(returns#118)#152, sum(profit#119)#153] Results [5]: [null AS channel#154, null AS id#155, sum(sales#117)#151 AS sales#156, sum(returns#118)#152 AS returns#157, sum(profit#119)#153 AS profit#158] -(114) Union +(94) Union -(115) HashAggregate [codegen id : 75] +(95) HashAggregate [codegen id : 75] Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] Functions: [] Aggregate Attributes: [] Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -(116) RowToColumnar +(96) Exchange Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Arguments: hashpartitioning(channel#31, id#32, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(117) CometColumnarExchange -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: hashpartitioning(channel#31, id#32, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(118) CometHashAggregate +(97) HashAggregate [codegen id : 76] Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -(119) CometTakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#114,returns#115,profit#116]), 100, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#114, returns#115, profit#116] - -(120) ColumnarToRow [codegen id : 76] +(98) TakeOrderedAndProject Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Arguments: 100, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#114, returns#115, profit#116] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (125) -+- * ColumnarToRow (124) - +- CometProject (123) - +- CometFilter (122) - +- CometScan parquet spark_catalog.default.date_dim (121) +BroadcastExchange (103) ++- * ColumnarToRow (102) + +- CometProject (101) + +- CometFilter (100) + +- CometScan parquet spark_catalog.default.date_dim (99) (unknown) Scan parquet spark_catalog.default.date_dim @@ -690,29 +604,29 @@ 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 -(122) CometFilter +(100) CometFilter Input [2]: [d_date_sk#6, d_date#159] Condition : (((isnotnull(d_date#159) AND (d_date#159 >= 1998-08-04)) AND (d_date#159 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(123) CometProject +(101) CometProject Input [2]: [d_date_sk#6, d_date#159] Arguments: [d_date_sk#6], [d_date_sk#6] -(124) ColumnarToRow [codegen id : 1] +(102) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(125) BroadcastExchange +(103) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 46 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 40 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 75 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 65 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index d27e3fd80c..670a7e6c3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -1,190 +1,168 @@ -WholeStageCodegen (76) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - RowToColumnar - WholeStageCodegen (75) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (24) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - RowToColumnar - WholeStageCodegen (23) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #3 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan 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) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (76) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Exchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (75) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (24) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel,id] #2 + WholeStageCodegen (23) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (8) + Project [s_store_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + InputAdapter + Exchange [s_store_sk] #3 + WholeStageCodegen (3) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan 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) ColumnarToRow InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - ColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - CometColumnarExchange [s_store_sk] #7 - RowToColumnar - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [sr_store_sk] - CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #9 - RowToColumnar - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange #10 - RowToColumnar - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #11 - RowToColumnar - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_web_page_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 + Exchange [s_store_sk] #7 + WholeStageCodegen (6) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometFilter [sr_store_sk] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (14) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #14 - RowToColumnar - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [wr_web_page_sk] - CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel] #15 - RowToColumnar - WholeStageCodegen (48) - 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] - ColumnarToRow + Exchange [cs_call_center_sk] #9 + WholeStageCodegen (10) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange #16 - RowToColumnar - WholeStageCodegen (73) - 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] - ColumnarToRow + Exchange #10 + WholeStageCodegen (13) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (22) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + Exchange [wp_web_page_sk] #11 + WholeStageCodegen (17) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (21) + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + Exchange [wp_web_page_sk] #14 + WholeStageCodegen (20) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_web_page_sk] + CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [wp_web_page_sk] #12 + WholeStageCodegen (49) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel] #15 + WholeStageCodegen (48) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (74) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange #16 + WholeStageCodegen (73) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index 5e806870bb..da7812b316 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -1,77 +1,74 @@ == Physical Plan == -TakeOrderedAndProject (73) -+- * Project (72) - +- * SortMergeJoin Inner (71) - :- * Project (47) - : +- * SortMergeJoin Inner (46) - : :- * Sort (22) - : : +- * HashAggregate (21) - : : +- * ColumnarToRow (20) - : : +- CometColumnarExchange (19) - : : +- RowToColumnar (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometColumnarExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometColumnarExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (14) - : +- * Sort (45) - : +- * Filter (44) - : +- * HashAggregate (43) - : +- * ColumnarToRow (42) - : +- CometColumnarExchange (41) - : +- RowToColumnar (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * ColumnarToRow (35) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometSortMergeJoin (32) - : : :- CometSort (26) - : : : +- CometColumnarExchange (25) - : : : +- CometFilter (24) - : : : +- CometScan parquet spark_catalog.default.web_sales (23) - : : +- CometSort (31) - : : +- CometColumnarExchange (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (36) - +- * Sort (70) - +- * Filter (69) - +- * HashAggregate (68) - +- * ColumnarToRow (67) - +- CometColumnarExchange (66) - +- RowToColumnar (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) - : +- CometProject (59) - : +- CometFilter (58) - : +- CometSortMergeJoin (57) - : :- CometSort (51) - : : +- CometColumnarExchange (50) - : : +- CometFilter (49) - : : +- CometScan parquet spark_catalog.default.catalog_sales (48) - : +- CometSort (56) - : +- CometColumnarExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan parquet spark_catalog.default.catalog_returns (52) - +- ReusedExchange (61) +TakeOrderedAndProject (70) ++- * Project (69) + +- * SortMergeJoin Inner (68) + :- * Project (45) + : +- * SortMergeJoin Inner (44) + : :- * Sort (21) + : : +- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Project (14) + : : : +- * Filter (13) + : : : +- * SortMergeJoin LeftOuter (12) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- * Sort (11) + : : : +- Exchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometProject (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : +- ReusedExchange (15) + : +- * Sort (43) + : +- * Filter (42) + : +- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (35) + : : +- * Filter (34) + : : +- * SortMergeJoin LeftOuter (33) + : : :- * Sort (26) + : : : +- Exchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.web_returns (27) + : +- ReusedExchange (36) + +- * Sort (67) + +- * Filter (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) + :- * Project (59) + : +- * Filter (58) + : +- * SortMergeJoin LeftOuter (57) + : :- * Sort (50) + : : +- Exchange (49) + : : +- * ColumnarToRow (48) + : : +- CometFilter (47) + : : +- CometScan parquet spark_catalog.default.catalog_sales (46) + : +- * Sort (56) + : +- Exchange (55) + : +- * ColumnarToRow (54) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.catalog_returns (51) + +- ReusedExchange (60) (unknown) Scan parquet spark_catalog.default.store_sales @@ -86,13 +83,16 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) -(7) CometProject +(8) CometProject Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] -(8) CometColumnarExchange +(9) ColumnarToRow [codegen id : 3] +Input [2]: [sr_item_sk#9, sr_ticket_number#10] + +(10) Exchange Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(9) CometSort +(11) Sort [codegen id : 4] Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] +Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter +(12) SortMergeJoin [codegen id : 6] +Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] +Join type: LeftOuter +Join condition: None -(11) CometFilter +(13) Filter [codegen id : 6] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Condition : isnull(sr_ticket_number#10) -(12) CometProject +(14) Project [codegen id : 6] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(14) ReusedExchange [Reuses operator id: 77] +(15) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#12, d_year#13] -(15) BroadcastHashJoin [codegen id : 2] +(16) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(16) Project [codegen id : 2] +(17) Project [codegen id : 6] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -(17) HashAggregate [codegen id : 2] +(18) HashAggregate [codegen id : 6] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum#14, sum#15, sum#16] Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(18) RowToColumnar -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] - -(19) CometColumnarExchange +(19) Exchange Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) ColumnarToRow [codegen id : 3] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] - -(21) HashAggregate [codegen id : 3] +(20) HashAggregate [codegen id : 7] Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(22) Sort [codegen id : 3] +(21) Sort [codegen id : 7] Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 @@ -182,17 +177,20 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(24) CometFilter +(23) CometFilter Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) -(25) CometColumnarExchange +(24) ColumnarToRow [codegen id : 8] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] + +(25) Exchange Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(26) CometSort +(26) Sort [codegen id : 9] Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] +Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_returns Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] @@ -209,82 +207,77 @@ Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] -(30) CometColumnarExchange +(30) ColumnarToRow [codegen id : 10] Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(31) CometSort +(31) Exchange Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(32) CometSortMergeJoin -Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Right output [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter +(32) Sort [codegen id : 11] +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 + +(33) SortMergeJoin [codegen id : 13] +Left keys [2]: [ws_order_number#29, ws_item_sk#27] +Right keys [2]: [wr_order_number#36, wr_item_sk#35] +Join type: LeftOuter +Join condition: None -(33) CometFilter +(34) Filter [codegen id : 13] Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] Condition : isnull(wr_order_number#36) -(34) CometProject +(35) Project [codegen id : 13] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -(35) ColumnarToRow [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] - -(36) ReusedExchange [Reuses operator id: 77] +(36) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#38, d_year#39] -(37) BroadcastHashJoin [codegen id : 5] +(37) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#33] Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(38) Project [codegen id : 5] +(38) Project [codegen id : 13] Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] -(39) HashAggregate [codegen id : 5] +(39) HashAggregate [codegen id : 13] Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum#40, sum#41, sum#42] Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -(40) RowToColumnar -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(41) CometColumnarExchange -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) ColumnarToRow [codegen id : 6] +(40) Exchange Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(43) HashAggregate [codegen id : 6] +(41) HashAggregate [codegen id : 14] Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] -(44) Filter [codegen id : 6] +(42) Filter [codegen id : 14] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Condition : (coalesce(ws_qty#51, 0) > 0) -(45) Sort [codegen id : 6] +(43) Sort [codegen id : 14] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 -(46) SortMergeJoin [codegen id : 7] +(44) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] Join type: Inner Join condition: None -(47) Project [codegen id : 7] +(45) Project [codegen id : 15] Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] @@ -296,17 +289,20 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(49) CometFilter +(47) CometFilter Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) -(50) CometColumnarExchange +(48) ColumnarToRow [codegen id : 16] +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] + +(49) Exchange Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(51) CometSort +(50) Sort [codegen id : 17] Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] +Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] @@ -315,104 +311,99 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(53) CometFilter +(52) CometFilter Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) -(54) CometProject +(53) CometProject Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] -(55) CometColumnarExchange +(54) ColumnarToRow [codegen id : 18] Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(56) CometSort +(55) Exchange Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(57) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Right output [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter +(56) Sort [codegen id : 19] +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 -(58) CometFilter +(57) SortMergeJoin [codegen id : 21] +Left keys [2]: [cs_order_number#56, cs_item_sk#55] +Right keys [2]: [cr_order_number#63, cr_item_sk#62] +Join type: LeftOuter +Join condition: None + +(58) Filter [codegen id : 21] Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] Condition : isnull(cr_order_number#63) -(59) CometProject +(59) Project [codegen id : 21] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(60) ColumnarToRow [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] - -(61) ReusedExchange [Reuses operator id: 77] +(60) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#65, d_year#66] -(62) BroadcastHashJoin [codegen id : 9] +(61) BroadcastHashJoin [codegen id : 21] Left keys [1]: [cs_sold_date_sk#60] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(63) Project [codegen id : 9] +(62) Project [codegen id : 21] Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] -(64) HashAggregate [codegen id : 9] +(63) HashAggregate [codegen id : 21] Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum#67, sum#68, sum#69] Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -(65) RowToColumnar -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] - -(66) CometColumnarExchange -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(67) ColumnarToRow [codegen id : 10] +(64) Exchange Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(68) HashAggregate [codegen id : 10] +(65) HashAggregate [codegen id : 22] Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] -(69) Filter [codegen id : 10] +(66) Filter [codegen id : 22] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Condition : (coalesce(cs_qty#78, 0) > 0) -(70) Sort [codegen id : 10] +(67) Sort [codegen id : 22] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 -(71) SortMergeJoin [codegen id : 11] +(68) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] Join type: Inner Join condition: None -(72) Project [codegen id : 11] +(69) Project [codegen id : 23] Output [13]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -(73) TakeOrderedAndProject +(70) TakeOrderedAndProject Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Arguments: 100, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (77) -+- * ColumnarToRow (76) - +- CometFilter (75) - +- CometScan parquet spark_catalog.default.date_dim (74) +BroadcastExchange (74) ++- * ColumnarToRow (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) (unknown) Scan parquet spark_catalog.default.date_dim @@ -422,19 +413,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(75) CometFilter +(72) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(76) ColumnarToRow [codegen id : 1] +(73) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(77) BroadcastExchange +(74) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 48 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index 4af11caa06..49bd173f67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -1,103 +1,127 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (11) + WholeStageCodegen (23) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] InputAdapter - WholeStageCodegen (7) + WholeStageCodegen (15) Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] InputAdapter - WholeStageCodegen (3) + WholeStageCodegen (7) Sort [ss_sold_year,ss_item_sk,ss_customer_sk] HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 - RowToColumnar - WholeStageCodegen (2) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow + InputAdapter + Exchange [d_year,ss_item_sk,ss_customer_sk] #1 + WholeStageCodegen (6) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + Filter [sr_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [sr_ticket_number] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_ticket_number,sr_item_sk] - CometColumnarExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (14) Sort [ws_sold_year,ws_item_sk,ws_customer_sk] Filter [ws_qty] HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow + InputAdapter + Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + WholeStageCodegen (13) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + Filter [wr_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [wr_order_number] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #6 - CometFilter [ws_item_sk,ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_order_number,wr_item_sk] - CometColumnarExchange [wr_order_number,wr_item_sk] #7 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + WholeStageCodegen (9) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #6 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (11) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #7 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter - WholeStageCodegen (10) + WholeStageCodegen (22) Sort [cs_sold_year,cs_item_sk,cs_customer_sk] Filter [cs_qty] HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - RowToColumnar - WholeStageCodegen (9) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow + InputAdapter + Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 + WholeStageCodegen (21) + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + Filter [cr_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (17) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #9 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cr_order_number] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #9 - CometFilter [cs_item_sk,cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_order_number,cr_item_sk] - CometColumnarExchange [cr_order_number,cr_item_sk] #10 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + WholeStageCodegen (19) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #10 + WholeStageCodegen (18) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 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 f9688675ba..0c9c978122 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,137 +1,124 @@ == Physical Plan == -* ColumnarToRow (133) -+- CometTakeOrderedAndProject (132) - +- CometHashAggregate (131) - +- CometColumnarExchange (130) - +- RowToColumnar (129) - +- * HashAggregate (128) - +- Union (127) - :- * HashAggregate (110) - : +- * ColumnarToRow (109) - : +- CometColumnarExchange (108) - : +- RowToColumnar (107) - : +- * HashAggregate (106) - : +- Union (105) - : :- * HashAggregate (40) - : : +- * ColumnarToRow (39) - : : +- CometColumnarExchange (38) - : : +- RowToColumnar (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * Project (21) - : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometColumnarExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometColumnarExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : : : : +- ReusedExchange (13) - : : : : +- BroadcastExchange (19) - : : : : +- * ColumnarToRow (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.store (16) - : : : +- BroadcastExchange (26) - : : : +- * ColumnarToRow (25) - : : : +- CometProject (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.item (22) - : : +- BroadcastExchange (33) - : : +- * ColumnarToRow (32) - : : +- CometProject (31) - : : +- CometFilter (30) - : : +- CometScan parquet spark_catalog.default.promotion (29) - : :- * HashAggregate (72) - : : +- * ColumnarToRow (71) - : : +- CometColumnarExchange (70) - : : +- RowToColumnar (69) - : : +- * HashAggregate (68) - : : +- * Project (67) - : : +- * BroadcastHashJoin Inner BuildRight (66) - : : :- * Project (64) - : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : :- * Project (61) - : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : :- * Project (55) - : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : :- * ColumnarToRow (52) - : : : : : : +- CometProject (51) - : : : : : : +- CometSortMergeJoin (50) - : : : : : : :- CometSort (44) - : : : : : : : +- CometColumnarExchange (43) - : : : : : : : +- CometFilter (42) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- CometSort (49) - : : : : : : +- CometColumnarExchange (48) - : : : : : : +- CometProject (47) - : : : : : : +- CometFilter (46) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) - : : : : : +- ReusedExchange (53) - : : : : +- BroadcastExchange (59) - : : : : +- * ColumnarToRow (58) - : : : : +- CometFilter (57) - : : : : +- CometScan parquet spark_catalog.default.catalog_page (56) - : : : +- ReusedExchange (62) - : : +- ReusedExchange (65) - : +- * HashAggregate (104) - : +- * ColumnarToRow (103) - : +- CometColumnarExchange (102) - : +- RowToColumnar (101) - : +- * HashAggregate (100) - : +- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- * Project (96) - : : +- * BroadcastHashJoin Inner BuildRight (95) - : : :- * Project (93) - : : : +- * BroadcastHashJoin Inner BuildRight (92) - : : : :- * Project (87) - : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : :- * ColumnarToRow (84) - : : : : : +- CometProject (83) - : : : : : +- CometSortMergeJoin (82) - : : : : : :- CometSort (76) - : : : : : : +- CometColumnarExchange (75) - : : : : : : +- CometFilter (74) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (73) - : : : : : +- CometSort (81) - : : : : : +- CometColumnarExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan parquet spark_catalog.default.web_returns (77) - : : : : +- ReusedExchange (85) - : : : +- BroadcastExchange (91) - : : : +- * ColumnarToRow (90) - : : : +- CometFilter (89) - : : : +- CometScan parquet spark_catalog.default.web_site (88) - : : +- ReusedExchange (94) - : +- ReusedExchange (97) - :- * HashAggregate (118) - : +- * ColumnarToRow (117) - : +- CometColumnarExchange (116) - : +- RowToColumnar (115) - : +- * HashAggregate (114) - : +- * HashAggregate (113) - : +- * ColumnarToRow (112) - : +- ReusedExchange (111) - +- * HashAggregate (126) - +- * ColumnarToRow (125) - +- CometColumnarExchange (124) - +- RowToColumnar (123) - +- * HashAggregate (122) - +- * HashAggregate (121) - +- * ColumnarToRow (120) - +- ReusedExchange (119) +TakeOrderedAndProject (120) ++- * HashAggregate (119) + +- Exchange (118) + +- * HashAggregate (117) + +- Union (116) + :- * HashAggregate (105) + : +- Exchange (104) + : +- * HashAggregate (103) + : +- Union (102) + : :- * HashAggregate (39) + : : +- Exchange (38) + : : +- * HashAggregate (37) + : : +- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Project (29) + : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : :- * Project (22) + : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : :- * Project (16) + : : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : : :- * Project (13) + : : : : : : +- * SortMergeJoin LeftOuter (12) + : : : : : : :- * Sort (5) + : : : : : : : +- Exchange (4) + : : : : : : : +- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- * Sort (11) + : : : : : : +- Exchange (10) + : : : : : : +- * ColumnarToRow (9) + : : : : : : +- CometProject (8) + : : : : : : +- CometFilter (7) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : : : +- ReusedExchange (14) + : : : : +- BroadcastExchange (20) + : : : : +- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.store (17) + : : : +- BroadcastExchange (27) + : : : +- * ColumnarToRow (26) + : : : +- CometProject (25) + : : : +- CometFilter (24) + : : : +- CometScan parquet spark_catalog.default.item (23) + : : +- BroadcastExchange (34) + : : +- * ColumnarToRow (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.promotion (30) + : :- * HashAggregate (70) + : : +- Exchange (69) + : : +- * HashAggregate (68) + : : +- * Project (67) + : : +- * BroadcastHashJoin Inner BuildRight (66) + : : :- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : :- * Project (61) + : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : :- * Project (55) + : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : :- * Project (52) + : : : : : : +- * SortMergeJoin LeftOuter (51) + : : : : : : :- * Sort (44) + : : : : : : : +- Exchange (43) + : : : : : : : +- * ColumnarToRow (42) + : : : : : : : +- CometFilter (41) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) + : : : : : : +- * Sort (50) + : : : : : : +- Exchange (49) + : : : : : : +- * ColumnarToRow (48) + : : : : : : +- CometProject (47) + : : : : : : +- CometFilter (46) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) + : : : : : +- ReusedExchange (53) + : : : : +- BroadcastExchange (59) + : : : : +- * ColumnarToRow (58) + : : : : +- CometFilter (57) + : : : : +- CometScan parquet spark_catalog.default.catalog_page (56) + : : : +- ReusedExchange (62) + : : +- ReusedExchange (65) + : +- * HashAggregate (101) + : +- Exchange (100) + : +- * HashAggregate (99) + : +- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * Project (92) + : : : +- * BroadcastHashJoin Inner BuildRight (91) + : : : :- * Project (86) + : : : : +- * BroadcastHashJoin Inner BuildRight (85) + : : : : :- * Project (83) + : : : : : +- * SortMergeJoin LeftOuter (82) + : : : : : :- * Sort (75) + : : : : : : +- Exchange (74) + : : : : : : +- * ColumnarToRow (73) + : : : : : : +- CometFilter (72) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) + : : : : : +- * Sort (81) + : : : : : +- Exchange (80) + : : : : : +- * ColumnarToRow (79) + : : : : : +- CometProject (78) + : : : : : +- CometFilter (77) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (76) + : : : : +- ReusedExchange (84) + : : : +- BroadcastExchange (90) + : : : +- * ColumnarToRow (89) + : : : +- CometFilter (88) + : : : +- CometScan parquet spark_catalog.default.web_site (87) + : : +- ReusedExchange (93) + : +- ReusedExchange (96) + :- * HashAggregate (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- ReusedExchange (106) + +- * HashAggregate (115) + +- Exchange (114) + +- * HashAggregate (113) + +- * HashAggregate (112) + +- ReusedExchange (111) (unknown) Scan parquet spark_catalog.default.store_sales @@ -146,13 +133,16 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) -(7) CometProject +(8) CometProject Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(8) CometColumnarExchange +(9) ColumnarToRow [codegen id : 3] Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(9) CometSort +(10) Exchange Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter +(11) Sort [codegen id : 4] +Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(12) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] +Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] +Join type: LeftOuter +Join condition: None -(12) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(13) Project [codegen id : 9] +Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(13) ReusedExchange [Reuses operator id: 138] +(14) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#14] -(14) BroadcastHashJoin [codegen id : 5] +(15) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(15) Project [codegen id : 5] +(16) Project [codegen id : 9] Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] @@ -209,24 +200,24 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(17) CometFilter +(18) CometFilter Input [2]: [s_store_sk#15, s_store_id#16] Condition : isnotnull(s_store_sk#15) -(18) ColumnarToRow [codegen id : 2] +(19) ColumnarToRow [codegen id : 6] Input [2]: [s_store_sk#15, s_store_id#16] -(19) BroadcastExchange +(20) BroadcastExchange Input [2]: [s_store_sk#15, s_store_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(20) BroadcastHashJoin [codegen id : 5] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(21) Project [codegen id : 5] +(22) Project [codegen id : 9] Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] @@ -237,28 +228,28 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(24) CometFilter Input [2]: [i_item_sk#17, i_current_price#18] Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) -(24) CometProject +(25) CometProject Input [2]: [i_item_sk#17, i_current_price#18] Arguments: [i_item_sk#17], [i_item_sk#17] -(25) ColumnarToRow [codegen id : 3] +(26) ColumnarToRow [codegen id : 7] Input [1]: [i_item_sk#17] -(26) BroadcastExchange +(27) BroadcastExchange Input [1]: [i_item_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) BroadcastHashJoin [codegen id : 5] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(28) Project [codegen id : 5] +(29) Project [codegen id : 9] Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] @@ -269,49 +260,43 @@ Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] ReadSchema: struct -(30) CometFilter +(31) CometFilter Input [2]: [p_promo_sk#19, p_channel_tv#20] Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) -(31) CometProject +(32) CometProject Input [2]: [p_promo_sk#19, p_channel_tv#20] Arguments: [p_promo_sk#19], [p_promo_sk#19] -(32) ColumnarToRow [codegen id : 4] +(33) ColumnarToRow [codegen id : 8] Input [1]: [p_promo_sk#19] -(33) BroadcastExchange +(34) BroadcastExchange Input [1]: [p_promo_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 5] +(35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_promo_sk#3] Right keys [1]: [p_promo_sk#19] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(36) Project [codegen id : 9] Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] -(36) HashAggregate [codegen id : 5] +(37) HashAggregate [codegen id : 9] Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Keys [1]: [s_store_id#16] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -(37) RowToColumnar +(38) Exchange Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] +Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(38) CometColumnarExchange -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(39) ColumnarToRow [codegen id : 6] -Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] - -(40) HashAggregate [codegen id : 6] +(39) HashAggregate [codegen id : 10] Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] Keys [1]: [s_store_id#16] 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)))] @@ -326,17 +311,20 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(42) CometFilter +(41) CometFilter Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) -(43) CometColumnarExchange +(42) ColumnarToRow [codegen id : 11] Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(44) CometSort +(43) Exchange Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45], [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST] +Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(44) Sort [codegen id : 12] +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] @@ -353,36 +341,37 @@ Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -(48) CometColumnarExchange +(48) ColumnarToRow [codegen id : 13] +Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] + +(49) Exchange Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(49) CometSort +(50) Sort [codegen id : 14] Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST] +Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 -(50) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Right output [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cs_item_sk#40, cs_order_number#42], [cr_item_sk#47, cr_order_number#48], LeftOuter +(51) SortMergeJoin [codegen id : 19] +Left keys [2]: [cs_item_sk#40, cs_order_number#42] +Right keys [2]: [cr_item_sk#47, cr_order_number#48] +Join type: LeftOuter +Join condition: None -(51) CometProject +(52) Project [codegen id : 19] +Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50], [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -(52) ColumnarToRow [codegen id : 11] -Input [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] - -(53) ReusedExchange [Reuses operator id: 138] +(53) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#52] -(54) BroadcastHashJoin [codegen id : 11] +(54) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#45] Right keys [1]: [d_date_sk#52] Join type: Inner Join condition: None -(55) Project [codegen id : 11] +(55) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] @@ -397,67 +386,61 @@ ReadSchema: struct Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Condition : isnotnull(cp_catalog_page_sk#53) -(58) ColumnarToRow [codegen id : 8] +(58) ColumnarToRow [codegen id : 16] Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] (59) BroadcastExchange Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 11] +(60) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#39] Right keys [1]: [cp_catalog_page_sk#53] Join type: Inner Join condition: None -(61) Project [codegen id : 11] +(61) Project [codegen id : 19] Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] -(62) ReusedExchange [Reuses operator id: 26] +(62) ReusedExchange [Reuses operator id: 27] Output [1]: [i_item_sk#55] -(63) BroadcastHashJoin [codegen id : 11] +(63) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#40] Right keys [1]: [i_item_sk#55] Join type: Inner Join condition: None -(64) Project [codegen id : 11] +(64) Project [codegen id : 19] Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] -(65) ReusedExchange [Reuses operator id: 33] +(65) ReusedExchange [Reuses operator id: 34] Output [1]: [p_promo_sk#56] -(66) BroadcastHashJoin [codegen id : 11] +(66) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#41] Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(67) Project [codegen id : 11] +(67) Project [codegen id : 19] Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] -(68) HashAggregate [codegen id : 11] +(68) HashAggregate [codegen id : 19] Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -(69) RowToColumnar -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] - -(70) CometColumnarExchange -Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(71) ColumnarToRow [codegen id : 12] +(69) Exchange Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(72) HashAggregate [codegen id : 12] +(70) HashAggregate [codegen id : 20] Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] @@ -472,17 +455,20 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(74) CometFilter +(72) CometFilter Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) -(75) CometColumnarExchange +(73) ColumnarToRow [codegen id : 21] Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(76) CometSort +(74) Exchange Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81], [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST] +Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(75) Sort [codegen id : 22] +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] @@ -491,44 +477,45 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(78) CometFilter +(77) CometFilter Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) -(79) CometProject +(78) CometProject Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -(80) CometColumnarExchange +(79) ColumnarToRow [codegen id : 23] +Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] + +(80) Exchange Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(81) CometSort +(81) Sort [codegen id : 24] Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST] +Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 -(82) CometSortMergeJoin -Left output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Right output [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [ws_item_sk#75, ws_order_number#78], [wr_item_sk#83, wr_order_number#84], LeftOuter +(82) SortMergeJoin [codegen id : 29] +Left keys [2]: [ws_item_sk#75, ws_order_number#78] +Right keys [2]: [wr_item_sk#83, wr_order_number#84] +Join type: LeftOuter +Join condition: None -(83) CometProject +(83) Project [codegen id : 29] +Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86], [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] - -(84) ColumnarToRow [codegen id : 17] -Input [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] -(85) ReusedExchange [Reuses operator id: 138] +(84) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#88] -(86) BroadcastHashJoin [codegen id : 17] +(85) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_sold_date_sk#81] Right keys [1]: [d_date_sk#88] Join type: Inner Join condition: None -(87) Project [codegen id : 17] +(86) Project [codegen id : 29] Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] @@ -539,213 +526,179 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(89) CometFilter +(88) CometFilter Input [2]: [web_site_sk#89, web_site_id#90] Condition : isnotnull(web_site_sk#89) -(90) ColumnarToRow [codegen id : 14] +(89) ColumnarToRow [codegen id : 26] Input [2]: [web_site_sk#89, web_site_id#90] -(91) BroadcastExchange +(90) BroadcastExchange Input [2]: [web_site_sk#89, web_site_id#90] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(92) BroadcastHashJoin [codegen id : 17] +(91) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_web_site_sk#76] Right keys [1]: [web_site_sk#89] Join type: Inner Join condition: None -(93) Project [codegen id : 17] +(92) Project [codegen id : 29] Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] -(94) ReusedExchange [Reuses operator id: 26] +(93) ReusedExchange [Reuses operator id: 27] Output [1]: [i_item_sk#91] -(95) BroadcastHashJoin [codegen id : 17] +(94) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_item_sk#75] Right keys [1]: [i_item_sk#91] Join type: Inner Join condition: None -(96) Project [codegen id : 17] +(95) Project [codegen id : 29] Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] -(97) ReusedExchange [Reuses operator id: 33] +(96) ReusedExchange [Reuses operator id: 34] Output [1]: [p_promo_sk#92] -(98) BroadcastHashJoin [codegen id : 17] +(97) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_promo_sk#77] Right keys [1]: [p_promo_sk#92] Join type: Inner Join condition: None -(99) Project [codegen id : 17] +(98) Project [codegen id : 29] Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] -(100) HashAggregate [codegen id : 17] +(99) HashAggregate [codegen id : 29] Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Keys [1]: [web_site_id#90] Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -(101) RowToColumnar +(100) Exchange Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(102) CometColumnarExchange -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(103) ColumnarToRow [codegen id : 18] -Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] - -(104) HashAggregate [codegen id : 18] +(101) HashAggregate [codegen id : 30] Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] Keys [1]: [web_site_id#90] Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] Results [5]: [web channel AS channel#106, concat(web_site, web_site_id#90) AS id#107, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#108, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#109, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#110] -(105) Union +(102) Union -(106) HashAggregate [codegen id : 19] +(103) HashAggregate [codegen id : 31] Input [5]: [channel#34, id#35, sales#36, returns#37, profit#38] Keys [2]: [channel#34, id#35] Functions [3]: [partial_sum(sales#36), partial_sum(returns#37), partial_sum(profit#38)] Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] Results [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -(107) RowToColumnar +(104) Exchange Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(108) CometColumnarExchange -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(109) ColumnarToRow [codegen id : 20] -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(110) HashAggregate [codegen id : 20] +(105) HashAggregate [codegen id : 32] Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] Results [5]: [channel#34, id#35, cast(sum(sales#36)#123 as decimal(37,2)) AS sales#126, cast(sum(returns#37)#124 as decimal(38,2)) AS returns#127, cast(sum(profit#38)#125 as decimal(38,2)) AS profit#128] -(111) ReusedExchange [Reuses operator id: 108] +(106) ReusedExchange [Reuses operator id: 104] Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -(112) ColumnarToRow [codegen id : 40] -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(113) HashAggregate [codegen id : 40] +(107) HashAggregate [codegen id : 64] Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] Results [4]: [channel#34, sum(sales#36)#123 AS sales#129, sum(returns#37)#124 AS returns#130, sum(profit#38)#125 AS profit#131] -(114) HashAggregate [codegen id : 40] +(108) HashAggregate [codegen id : 64] Input [4]: [channel#34, sales#129, returns#130, profit#131] Keys [1]: [channel#34] Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] Results [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -(115) RowToColumnar -Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(116) CometColumnarExchange -Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(117) ColumnarToRow [codegen id : 41] +(109) Exchange Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(118) HashAggregate [codegen id : 41] +(110) HashAggregate [codegen id : 65] Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Keys [1]: [channel#34] Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] Aggregate Attributes [3]: [sum(sales#129)#144, sum(returns#130)#145, sum(profit#131)#146] Results [5]: [channel#34, null AS id#147, sum(sales#129)#144 AS sales#148, sum(returns#130)#145 AS returns#149, sum(profit#131)#146 AS profit#150] -(119) ReusedExchange [Reuses operator id: 108] +(111) ReusedExchange [Reuses operator id: 104] Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -(120) ColumnarToRow [codegen id : 61] -Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(121) HashAggregate [codegen id : 61] +(112) HashAggregate [codegen id : 97] Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] Results [3]: [sum(sales#36)#123 AS sales#129, sum(returns#37)#124 AS returns#130, sum(profit#38)#125 AS profit#131] -(122) HashAggregate [codegen id : 61] +(113) HashAggregate [codegen id : 97] Input [3]: [sales#129, returns#130, profit#131] Keys: [] Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] Aggregate Attributes [6]: [sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] Results [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -(123) RowToColumnar -Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] - -(124) CometColumnarExchange -Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] - -(125) ColumnarToRow [codegen id : 62] +(114) Exchange Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] -(126) HashAggregate [codegen id : 62] +(115) HashAggregate [codegen id : 98] Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Keys: [] Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] Aggregate Attributes [3]: [sum(sales#129)#163, sum(returns#130)#164, sum(profit#131)#165] Results [5]: [null AS channel#166, null AS id#167, sum(sales#129)#163 AS sales#168, sum(returns#130)#164 AS returns#169, sum(profit#131)#165 AS profit#170] -(127) Union +(116) Union -(128) HashAggregate [codegen id : 63] +(117) HashAggregate [codegen id : 99] Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Functions: [] Aggregate Attributes: [] Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -(129) RowToColumnar +(118) Exchange Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Arguments: hashpartitioning(channel#34, id#35, sales#126, returns#127, profit#128, 5), ENSURE_REQUIREMENTS, [plan_id=18] -(130) CometColumnarExchange -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Arguments: hashpartitioning(channel#34, id#35, sales#126, returns#127, profit#128, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] - -(131) CometHashAggregate +(119) HashAggregate [codegen id : 100] Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -(132) CometTakeOrderedAndProject -Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#34 ASC NULLS FIRST,id#35 ASC NULLS FIRST], output=[channel#34,id#35,sales#126,returns#127,profit#128]), 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#126, returns#127, profit#128] - -(133) ColumnarToRow [codegen id : 64] +(120) TakeOrderedAndProject Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#126, returns#127, profit#128] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (138) -+- * ColumnarToRow (137) - +- CometProject (136) - +- CometFilter (135) - +- CometScan parquet spark_catalog.default.date_dim (134) +BroadcastExchange (125) ++- * ColumnarToRow (124) + +- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) (unknown) Scan parquet spark_catalog.default.date_dim @@ -755,23 +708,23 @@ 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 -(135) CometFilter +(122) CometFilter Input [2]: [d_date_sk#14, d_date#171] Condition : (((isnotnull(d_date#171) AND (d_date#171 >= 1998-08-04)) AND (d_date#171 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(136) CometProject +(123) CometProject Input [2]: [d_date_sk#14, d_date#171] Arguments: [d_date_sk#14], [d_date_sk#14] -(137) ColumnarToRow [codegen id : 1] +(124) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(138) BroadcastExchange +(125) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 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 cd7515a4d5..34e47dcbaf 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,193 +1,207 @@ -WholeStageCodegen (64) - ColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - RowToColumnar - WholeStageCodegen (63) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (20) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - RowToColumnar - WholeStageCodegen (19) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (100) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Exchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (99) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (32) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel,id] #2 + WholeStageCodegen (31) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (10) + 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] InputAdapter - Union - WholeStageCodegen (6) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - RowToColumnar - WholeStageCodegen (5) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - 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_ticket_number,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_ticket_number] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] - CometScan 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) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number] - CometColumnarExchange [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] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + Exchange [s_store_id] #3 + WholeStageCodegen (9) + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [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] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + WholeStageCodegen (2) + Sort [ss_item_sk,ss_ticket_number] + InputAdapter + Exchange [ss_item_sk,ss_ticket_number] #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometScan 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) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_item_sk,sr_ticket_number] #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (12) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #10 - RowToColumnar - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - 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_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometColumnarExchange [cs_item_sk,cs_order_number] #11 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - CometScan 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] - CometColumnarExchange [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] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk] #8 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (7) + ColumnarToRow InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (18) - 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] - ColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #14 - RowToColumnar - WholeStageCodegen (17) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - 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_order_number,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_order_number] - CometColumnarExchange [ws_item_sk,ws_order_number] #15 - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - CometScan 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] - CometColumnarExchange [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] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + CometProject [i_item_sk] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_channel_tv,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (20) + 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] + InputAdapter + Exchange [cp_catalog_page_id] #10 + WholeStageCodegen (19) + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [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] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (12) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #11 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometScan 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 InputAdapter - ReusedExchange [i_item_sk] #8 + WholeStageCodegen (14) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #12 + WholeStageCodegen (13) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (41) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [channel] #18 - RowToColumnar - WholeStageCodegen (40) - 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] - ColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (62) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange #19 - RowToColumnar - WholeStageCodegen (61) - 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] - ColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + InputAdapter + ReusedExchange [i_item_sk] #8 + InputAdapter + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (30) + 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] + InputAdapter + Exchange [web_site_id] #14 + WholeStageCodegen (29) + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [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] + SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + InputAdapter + WholeStageCodegen (22) + Sort [ws_item_sk,ws_order_number] + InputAdapter + Exchange [ws_item_sk,ws_order_number] #15 + WholeStageCodegen (21) + ColumnarToRow + InputAdapter + CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometScan 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 + InputAdapter + WholeStageCodegen (24) + Sort [wr_item_sk,wr_order_number] + InputAdapter + Exchange [wr_item_sk,wr_order_number] #16 + WholeStageCodegen (23) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (26) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + InputAdapter + ReusedExchange [i_item_sk] #8 + InputAdapter + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (65) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel] #18 + WholeStageCodegen (64) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (98) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange #19 + WholeStageCodegen (97) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt index c9db800a80..610ae89672 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt @@ -1,48 +1,38 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- Window (42) - +- * ColumnarToRow (41) - +- CometSort (40) - +- CometColumnarExchange (39) - +- CometHashAggregate (38) - +- CometColumnarExchange (37) - +- RowToColumnar (36) - +- * HashAggregate (35) - +- Union (34) - :- * HashAggregate (17) - : +- * ColumnarToRow (16) - : +- CometColumnarExchange (15) - : +- RowToColumnar (14) - : +- * HashAggregate (13) - : +- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - :- * HashAggregate (25) - : +- * ColumnarToRow (24) - : +- CometColumnarExchange (23) - : +- RowToColumnar (22) - : +- * HashAggregate (21) - : +- * HashAggregate (20) - : +- * ColumnarToRow (19) - : +- ReusedExchange (18) - +- * HashAggregate (33) - +- * ColumnarToRow (32) - +- CometColumnarExchange (31) - +- RowToColumnar (30) - +- * HashAggregate (29) - +- * HashAggregate (28) - +- * ColumnarToRow (27) - +- ReusedExchange (26) +TakeOrderedAndProject (34) ++- * Project (33) + +- Window (32) + +- * Sort (31) + +- Exchange (30) + +- * HashAggregate (29) + +- Exchange (28) + +- * HashAggregate (27) + +- Union (26) + :- * HashAggregate (15) + : +- Exchange (14) + : +- * HashAggregate (13) + : +- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + :- * HashAggregate (20) + : +- Exchange (19) + : +- * HashAggregate (18) + : +- * HashAggregate (17) + : +- ReusedExchange (16) + +- * HashAggregate (25) + +- Exchange (24) + +- * HashAggregate (23) + +- * HashAggregate (22) + +- ReusedExchange (21) (unknown) Scan parquet spark_catalog.default.web_sales @@ -60,7 +50,7 @@ Condition : isnotnull(ws_item_sk#1) (3) ColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(4) ReusedExchange [Reuses operator id: 49] +(4) ReusedExchange [Reuses operator id: 39] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 3] @@ -108,149 +98,121 @@ Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum#9] Results [3]: [i_category#8, i_class#7, sum#10] -(14) RowToColumnar +(14) Exchange Input [3]: [i_category#8, i_class#7, sum#10] +Arguments: hashpartitioning(i_category#8, i_class#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [3]: [i_category#8, i_class#7, sum#10] -Arguments: hashpartitioning(i_category#8, i_class#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 4] -Input [3]: [i_category#8, i_class#7, sum#10] - -(17) HashAggregate [codegen id : 4] +(15) HashAggregate [codegen id : 4] Input [3]: [i_category#8, i_class#7, sum#10] Keys [2]: [i_category#8, i_class#7] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) as decimal(27,2)) AS total_sum#12, i_category#8, i_class#7, 0 AS g_category#13, 0 AS g_class#14, 0 AS lochierarchy#15] -(18) ReusedExchange [Reuses operator id: 15] +(16) ReusedExchange [Reuses operator id: 14] Output [3]: [i_category#8, i_class#7, sum#16] -(19) ColumnarToRow [codegen id : 8] -Input [3]: [i_category#8, i_class#7, sum#16] - -(20) HashAggregate [codegen id : 8] +(17) HashAggregate [codegen id : 8] Input [3]: [i_category#8, i_class#7, sum#16] Keys [2]: [i_category#8, i_class#7] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) AS total_sum#17, i_category#8] -(21) HashAggregate [codegen id : 8] +(18) HashAggregate [codegen id : 8] Input [2]: [total_sum#17, i_category#8] Keys [1]: [i_category#8] Functions [1]: [partial_sum(total_sum#17)] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [3]: [i_category#8, sum#20, isEmpty#21] -(22) RowToColumnar +(19) Exchange Input [3]: [i_category#8, sum#20, isEmpty#21] +Arguments: hashpartitioning(i_category#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(23) CometColumnarExchange -Input [3]: [i_category#8, sum#20, isEmpty#21] -Arguments: hashpartitioning(i_category#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) ColumnarToRow [codegen id : 9] -Input [3]: [i_category#8, sum#20, isEmpty#21] - -(25) HashAggregate [codegen id : 9] +(20) HashAggregate [codegen id : 9] Input [3]: [i_category#8, sum#20, isEmpty#21] Keys [1]: [i_category#8] Functions [1]: [sum(total_sum#17)] Aggregate Attributes [1]: [sum(total_sum#17)#22] Results [6]: [sum(total_sum#17)#22 AS total_sum#23, i_category#8, null AS i_class#24, 0 AS g_category#25, 1 AS g_class#26, 1 AS lochierarchy#27] -(26) ReusedExchange [Reuses operator id: 15] +(21) ReusedExchange [Reuses operator id: 14] Output [3]: [i_category#8, i_class#7, sum#28] -(27) ColumnarToRow [codegen id : 13] -Input [3]: [i_category#8, i_class#7, sum#28] - -(28) HashAggregate [codegen id : 13] +(22) HashAggregate [codegen id : 13] Input [3]: [i_category#8, i_class#7, sum#28] Keys [2]: [i_category#8, i_class#7] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) AS total_sum#17] -(29) HashAggregate [codegen id : 13] +(23) HashAggregate [codegen id : 13] Input [1]: [total_sum#17] Keys: [] Functions [1]: [partial_sum(total_sum#17)] Aggregate Attributes [2]: [sum#29, isEmpty#30] Results [2]: [sum#31, isEmpty#32] -(30) RowToColumnar -Input [2]: [sum#31, isEmpty#32] - -(31) CometColumnarExchange -Input [2]: [sum#31, isEmpty#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(32) ColumnarToRow [codegen id : 14] +(24) Exchange Input [2]: [sum#31, isEmpty#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(33) HashAggregate [codegen id : 14] +(25) HashAggregate [codegen id : 14] Input [2]: [sum#31, isEmpty#32] Keys: [] Functions [1]: [sum(total_sum#17)] Aggregate Attributes [1]: [sum(total_sum#17)#33] Results [6]: [sum(total_sum#17)#33 AS total_sum#34, null AS i_category#35, null AS i_class#36, 1 AS g_category#37, 1 AS g_class#38, 2 AS lochierarchy#39] -(34) Union +(26) Union -(35) HashAggregate [codegen id : 15] +(27) HashAggregate [codegen id : 15] Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] Functions: [] Aggregate Attributes: [] Results [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -(36) RowToColumnar +(28) Exchange Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +Arguments: hashpartitioning(total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(37) CometColumnarExchange -Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -Arguments: hashpartitioning(total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(38) CometHashAggregate +(29) HashAggregate [codegen id : 16] Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] Functions: [] +Aggregate Attributes: [] +Results [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, CASE WHEN (g_class#14 = 0) THEN i_category#8 END AS _w0#40] -(39) CometColumnarExchange -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] -Arguments: hashpartitioning(lochierarchy#15, _w0#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(40) CometSort +(30) Exchange Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] -Arguments: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40], [lochierarchy#15 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#12 DESC NULLS LAST] +Arguments: hashpartitioning(lochierarchy#15, _w0#40, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) ColumnarToRow [codegen id : 16] +(31) Sort [codegen id : 17] Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] +Arguments: [lochierarchy#15 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#12 DESC NULLS LAST], false, 0 -(42) Window +(32) Window Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] Arguments: [rank(total_sum#12) windowspecdefinition(lochierarchy#15, _w0#40, total_sum#12 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [lochierarchy#15, _w0#40], [total_sum#12 DESC NULLS LAST] -(43) Project [codegen id : 17] +(33) Project [codegen id : 18] Output [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] Input [6]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40, rank_within_parent#41] -(44) TakeOrderedAndProject +(34) TakeOrderedAndProject Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (39) ++- * ColumnarToRow (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.date_dim (35) (unknown) Scan parquet spark_catalog.default.date_dim @@ -260,18 +222,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(36) CometFilter Input [2]: [d_date_sk#5, d_month_seq#42] Condition : (((isnotnull(d_month_seq#42) AND (d_month_seq#42 >= 1212)) AND (d_month_seq#42 <= 1223)) AND isnotnull(d_date_sk#5)) -(47) CometProject +(37) CometProject Input [2]: [d_date_sk#5, d_month_seq#42] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) ColumnarToRow [codegen id : 1] +(38) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(49) BroadcastExchange +(39) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index 6abc26081c..5c5e088857 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -1,74 +1,66 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (17) + WholeStageCodegen (18) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (16) - ColumnarToRow + WholeStageCodegen (17) + Sort [lochierarchy,_w0,total_sum] InputAdapter - CometSort [lochierarchy,_w0,total_sum] - CometColumnarExchange [lochierarchy,_w0] #1 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - RowToColumnar + Exchange [lochierarchy,_w0] #1 + WholeStageCodegen (16) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] [_w0] + InputAdapter + Exchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 WholeStageCodegen (15) HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] InputAdapter Union WholeStageCodegen (4) HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,g_category,g_class,lochierarchy,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 + InputAdapter + Exchange [i_category,i_class] #3 + WholeStageCodegen (3) + HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] + Project [ws_net_paid,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (9) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #6 - RowToColumnar - WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 + InputAdapter + Exchange [i_category] #6 + WholeStageCodegen (8) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 WholeStageCodegen (14) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - ColumnarToRow - InputAdapter - CometColumnarExchange #7 - RowToColumnar - WholeStageCodegen (13) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - ColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 + InputAdapter + Exchange #7 + WholeStageCodegen (13) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index 6553eeabe7..7fa138d5e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -1,31 +1,25 @@ == Physical Plan == -* ColumnarToRow (27) -+- CometSort (26) - +- CometColumnarExchange (25) - +- RowToColumnar (24) - +- * Project (23) - +- Window (22) - +- * ColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- RowToColumnar (18) - +- * HashAggregate (17) - +- * ColumnarToRow (16) - +- CometColumnarExchange (15) - +- RowToColumnar (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +* Sort (21) ++- Exchange (20) + +- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) (unknown) Scan parquet spark_catalog.default.store_sales @@ -71,7 +65,7 @@ Join condition: None Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 32] +(10) ReusedExchange [Reuses operator id: 26] Output [1]: [d_date_sk#11] (11) BroadcastHashJoin [codegen id : 3] @@ -91,67 +85,49 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#12] Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) RowToColumnar +(14) Exchange Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) CometColumnarExchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] - -(17) HashAggregate [codegen id : 4] +(15) HashAggregate [codegen id : 4] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16] -(18) RowToColumnar -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] - -(19) CometColumnarExchange +(16) Exchange Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) CometSort +(17) Sort [codegen id : 5] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16], [i_class#9 ASC NULLS FIRST] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(21) ColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] - -(22) Window +(18) Window Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(23) Project [codegen id : 6] +(19) Project [codegen id : 6] Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] -(24) RowToColumnar -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] - -(25) CometColumnarExchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometSort +(20) Exchange Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Arguments: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18], [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(27) ColumnarToRow [codegen id : 7] +(21) Sort [codegen id : 7] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (32) -+- * ColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan parquet spark_catalog.default.date_dim (28) +BroadcastExchange (26) ++- * ColumnarToRow (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) (unknown) Scan parquet spark_catalog.default.date_dim @@ -161,18 +137,18 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(29) CometFilter +(23) CometFilter Input [2]: [d_date_sk#11, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(30) CometProject +(24) CometProject Input [2]: [d_date_sk#11, d_date#19] Arguments: [d_date_sk#11], [d_date_sk#11] -(31) ColumnarToRow [codegen id : 1] +(25) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(32) BroadcastExchange +(26) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index ef8abe2979..b7489a0aff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -1,49 +1,43 @@ WholeStageCodegen (7) - ColumnarToRow + Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] InputAdapter - CometSort [i_category,i_class,i_item_id,i_item_desc,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - RowToColumnar - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometSort [i_class] - CometColumnarExchange [i_class] #2 - RowToColumnar - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - ColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - RowToColumnar - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (6) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (5) + Sort [i_class] + InputAdapter + Exchange [i_class] #2 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 From 65a95157d500fd96425e03d2830f1a0c08e118da Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 4 May 2024 22:06:11 -0700 Subject: [PATCH 26/30] Restore diff --- dev/diffs/3.4.2.diff | 128 +++++-------------------------------------- 1 file changed, 15 insertions(+), 113 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 86a28c2556..47ad71ded1 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -542,7 +542,7 @@ index 1792b4c32eb..1616e6f39bd 100644 assert(shuffleMergeJoins.size == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -index 5125708be32..123f58c522a 100644 +index 5125708be32..cff6252a36b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -29,6 +29,8 @@ import org.apache.spark.sql.catalyst.TableIdentifier @@ -564,23 +564,6 @@ index 5125708be32..123f58c522a 100644 withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "0", SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> "1") { -@@ -868,14 +871,14 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan - } - val executed = df.queryExecution.executedPlan - val executedJoins = collect(executed) { -- case j: SortMergeJoinExec => j -+ case j: CometSortMergeJoinExec => j - } - // This only applies to the above tested queries, in which a child SortMergeJoin always - // contains the SortOrder required by its parent SortMergeJoin. Thus, SortExec should never - // appear as parent of SortMergeJoin. - executed.foreach { - case s: SortExec => s.foreach { -- case j: SortMergeJoinExec => fail( -+ case j: CometSortMergeJoinExec => fail( - s"No extra sort should be added since $j already satisfies the required ordering" - ) - case _ => @@ -1114,9 +1117,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) .groupBy($"k1").count() @@ -1019,38 +1002,18 @@ index 47679ed7865..9ffbaecb98e 100644 assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -index ac710c32296..88a5329e74e 100644 +index ac710c32296..e163c1a6a76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} -+import org.apache.spark.sql.comet.{CometProjectExec, CometSortExec, CometSortMergeJoinExec} ++import org.apache.spark.sql.comet.CometSortMergeJoinExec import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -@@ -192,7 +193,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val joinUniqueDF = df1.join(df2.hint(hint), $"k1" === $"k2", "full_outer") - assert(joinUniqueDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true -- case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(_: CometSortMergeJoinExec))) -+ if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(joinUniqueDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4), - Row(null, 5), Row(null, 6), Row(null, 7), Row(null, 8), Row(null, 9))) -@@ -202,7 +204,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val joinNonUniqueDF = df1.join(df2.hint(hint), $"k1" === $"k2" % 3, "full_outer") - assert(joinNonUniqueDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true -- case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(_: CometSortMergeJoinExec))) -+ if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(joinNonUniqueDF, Seq(Row(0, 0), Row(0, 3), Row(0, 6), Row(0, 9), Row(1, 1), - Row(1, 4), Row(1, 7), Row(2, 2), Row(2, 5), Row(2, 8), Row(3, null), Row(4, null))) -@@ -224,6 +227,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -224,6 +225,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession assert(twoJoinsDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true @@ -1058,25 +1021,7 @@ index ac710c32296..88a5329e74e 100644 }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0, 0, 0), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, null), Row(4, 4, null), -@@ -240,7 +244,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - // test one left outer sort merge join - val oneLeftOuterJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_outer") - assert(oneLeftOuterJoinDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(_: CometSortMergeJoinExec))) => true - }.size === 1) - checkAnswer(oneLeftOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, null), - Row(5, null), Row(6, null), Row(7, null), Row(8, null), Row(9, null))) -@@ -248,7 +252,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - // test one right outer sort merge join - val oneRightOuterJoinDF = df2.join(df3.hint("SHUFFLE_MERGE"), $"k2" === $"k3", "right_outer") - assert(oneRightOuterJoinDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter(_: CometSortMergeJoinExec))) => true - }.size === 1) - checkAnswer(oneRightOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(null, 4), - Row(null, 5))) -@@ -258,6 +262,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -258,6 +260,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "right_outer") assert(twoJoinsDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : SortMergeJoinExec) => true @@ -1084,60 +1029,38 @@ index ac710c32296..88a5329e74e 100644 }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, null, 4), Row(5, null, 5), -@@ -272,7 +277,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - // test one left semi sort merge join - val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_semi") - assert(oneJoinDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true -+ case WholeStageCodegenExec( -+ ColumnarToRowExec(InputAdapter( -+ CometProjectExec(_, _, _, _, _: CometSortMergeJoinExec, _)))) => true - }.size === 1) - checkAnswer(oneJoinDF, Seq(Row(0), Row(1), Row(2), Row(3))) - -@@ -280,8 +287,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -280,8 +283,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | - WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: CometSortMergeJoinExec => true ++ case _: SortMergeJoinExec => true }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) } -@@ -294,7 +300,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - // test one left anti sort merge join - val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") - assert(oneJoinDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true -+ case WholeStageCodegenExec( -+ ColumnarToRowExec(InputAdapter( -+ CometProjectExec(_, _, _, _, _: CometSortMergeJoinExec, _)))) => true - }.size === 1) - checkAnswer(oneJoinDF, Seq(Row(4), Row(5), Row(6), Row(7), Row(8), Row(9))) - -@@ -302,8 +310,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -302,8 +304,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val twoJoinsDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") .join(df3.hint("SHUFFLE_MERGE"), $"k1" === $"k3", "left_anti") assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | - WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: CometSortMergeJoinExec => true ++ case _: SortMergeJoinExec => true }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(6), Row(7), Row(8), Row(9))) } -@@ -436,7 +443,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -436,7 +437,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val plan = df.queryExecution.executedPlan assert(plan.exists(p => p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec])) + p.asInstanceOf[WholeStageCodegenExec].collect { -+ case _: CometSortExec => true ++ case _: SortExec => true + }.nonEmpty)) assert(df.collect() === Array(Row(1), Row(2), Row(3))) } -@@ -616,7 +625,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -616,7 +619,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .write.mode(SaveMode.Overwrite).parquet(path) withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "255", @@ -1523,10 +1446,10 @@ index bd9c79e5b96..ab7584e768e 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala -index ce43edb79c1..89e05c75380 100644 +index ce43edb79c1..c414b19eda7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala -@@ -17,9 +17,10 @@ +@@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources @@ -1534,29 +1457,8 @@ index ce43edb79c1..89e05c75380 100644 +import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, NullsFirst, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Sort} -+import org.apache.spark.sql.comet.CometSortExec import org.apache.spark.sql.execution.{QueryExecution, SortExec} - import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec - import org.apache.spark.sql.internal.SQLConf -@@ -224,7 +225,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write - - // assert the outer most sort in the executed plan - assert(plan.collectFirst { -- case s: SortExec => s -+ case s: CometSortExec => s.originalPlan - }.exists { - case SortExec(Seq( - SortOrder(AttributeReference("key", IntegerType, _, _), Ascending, NullsFirst, _), -@@ -271,7 +272,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write - - // assert the outer most sort in the executed plan - assert(plan.collectFirst { -- case s: SortExec => s -+ case s: CometSortExec => s.originalPlan - }.exists { - case SortExec(Seq( - SortOrder(AttributeReference("value", StringType, _, _), Ascending, NullsFirst, _), -@@ -305,7 +306,8 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write +@@ -305,7 +305,8 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write } } From c9633baf8ecdce5f7cdf92d9688b6e919766a575 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 8 May 2024 11:26:45 -0700 Subject: [PATCH 27/30] Update core/src/execution/datafusion/planner.rs Co-authored-by: Andy Grove --- core/src/execution/datafusion/planner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index 958505a921..d38381fbbb 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -1046,7 +1046,7 @@ impl PhysicalPlanner { .iter() .map(|f| match f.data_type() { DataType::Dictionary(_, val_type) => { - Arc::new(Field::new(f.name(), val_type.as_ref().clone(), true)) + Arc::new(Field::new(f.name(), val_type.as_ref().clone(), f.is_nullable())) } _ => f.clone(), }) From a61dc700b424b77dae91a46f21a7ed0cda4d855f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 8 May 2024 11:33:40 -0700 Subject: [PATCH 28/30] Update core/src/execution/datafusion/planner.rs Co-authored-by: Andy Grove --- core/src/execution/datafusion/planner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index d38381fbbb..f4eae2be4d 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -1076,7 +1076,7 @@ impl PhysicalPlanner { // we need to change dictionary type to data type for join filter expression. .map(|f| match f.data_type() { DataType::Dictionary(_, val_type) => { - Field::new(f.name(), val_type.as_ref().clone(), true) + Field::new(f.name(), val_type.as_ref().clone(), f.is_nullable()) } _ => f.clone(), }) From cceb5525307bab2931d273bfea25129862508105 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 8 May 2024 11:43:41 -0700 Subject: [PATCH 29/30] Fix style --- core/src/execution/datafusion/planner.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index f4eae2be4d..d1fb252ccb 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -1045,9 +1045,11 @@ impl PhysicalPlanner { .fields() .iter() .map(|f| match f.data_type() { - DataType::Dictionary(_, val_type) => { - Arc::new(Field::new(f.name(), val_type.as_ref().clone(), f.is_nullable())) - } + DataType::Dictionary(_, val_type) => Arc::new(Field::new( + f.name(), + val_type.as_ref().clone(), + f.is_nullable(), + )), _ => f.clone(), }) .collect(); From cbc5305d4039ec7b75a3a438c1dd3a86df0dc05a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 8 May 2024 12:19:45 -0700 Subject: [PATCH 30/30] Use ShuffleExchangeLike instead --- dev/diffs/3.4.2.diff | 322 ++++++++++++++++++++++++------------------- 1 file changed, 179 insertions(+), 143 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 47ad71ded1..19bf6dd41b 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -211,44 +211,47 @@ index 0efe0877e9b..423d3b3d76d 100644 -- SELECT_HAVING -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala -index cf40e944c09..fc940f9452f 100644 +index cf40e944c09..bdd5be4f462 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala -@@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException - import org.apache.spark.sql.catalyst.expressions.SubqueryExpression - import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, Join, JoinStrategyHint, SHUFFLE_HASH} - import org.apache.spark.sql.catalyst.util.DateTimeConstants -+import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec +@@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, RDDScanExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar._ -@@ -516,7 +517,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike + import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate + import org.apache.spark.sql.functions._ + import org.apache.spark.sql.internal.SQLConf +@@ -516,7 +516,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils */ private def verifyNumExchanges(df: DataFrame, expected: Int): Unit = { assert( - collect(df.queryExecution.executedPlan) { case e: ShuffleExchangeExec => e }.size == expected) + collect(df.queryExecution.executedPlan) { -+ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => 1 }.size == expected) ++ case _: ShuffleExchangeLike => 1 }.size == expected) } test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala -index ea5e47ede55..cab59443c79 100644 +index ea5e47ede55..814b92d090f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala -@@ -24,6 +24,7 @@ import scala.util.Random - import org.scalatest.matchers.must.Matchers.the - - import org.apache.spark.SparkException -+import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec +@@ -27,7 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} -@@ -756,6 +757,7 @@ class DataFrameAggregateSuite extends QueryTest +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike + import org.apache.spark.sql.expressions.Window + import org.apache.spark.sql.functions._ + import org.apache.spark.sql.internal.SQLConf +@@ -755,7 +755,7 @@ class DataFrameAggregateSuite extends QueryTest + assert(objHashAggPlans.nonEmpty) val exchangePlans = collect(aggPlan) { - case shuffle: ShuffleExchangeExec => shuffle -+ case shuffle: CometShuffleExchangeExec => shuffle +- case shuffle: ShuffleExchangeExec => shuffle ++ case shuffle: ShuffleExchangeLike => shuffle } assert(exchangePlans.length == 1) } @@ -268,44 +271,55 @@ index 56e9520fdab..917932336df 100644 spark.range(100).write.saveAsTable(s"$dbName.$table2Name") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala -index 9ddb4abe98b..2bd28d4041d 100644 +index 9ddb4abe98b..1b9269acef1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala -@@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Attri - import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation - import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LeafNode, LocalRelation, LogicalPlan, OneRowRelation, Statistics} - import org.apache.spark.sql.catalyst.util.DateTimeUtils -+import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec - import org.apache.spark.sql.connector.FakeV2Provider +@@ -43,7 +43,7 @@ import org.apache.spark.sql.connector.FakeV2Provider import org.apache.spark.sql.execution.{FilterExec, LogicalRDD, QueryExecution, SortExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -@@ -2291,7 +2292,7 @@ class DataFrameSuite extends QueryTest + import org.apache.spark.sql.execution.aggregate.HashAggregateExec +-import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec, ShuffleExchangeLike} ++import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.expressions.{Aggregator, Window} + import org.apache.spark.sql.functions._ + import org.apache.spark.sql.internal.SQLConf +@@ -1981,7 +1981,7 @@ class DataFrameSuite extends QueryTest + fail("Should not have back to back Aggregates") + } + atFirstAgg = true +- case e: ShuffleExchangeExec => atFirstAgg = false ++ case e: ShuffleExchangeLike => atFirstAgg = false + case _ => + } + } +@@ -2291,7 +2291,7 @@ class DataFrameSuite extends QueryTest checkAnswer(join, df) assert( collect(join.queryExecution.executedPlan) { - case e: ShuffleExchangeExec => true }.size === 1) -+ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => true }.size === 1) ++ case _: ShuffleExchangeLike => true }.size === 1) assert( collect(join.queryExecution.executedPlan) { case e: ReusedExchangeExec => true }.size === 1) val broadcasted = broadcast(join) -@@ -2299,7 +2300,7 @@ class DataFrameSuite extends QueryTest +@@ -2299,7 +2299,7 @@ class DataFrameSuite extends QueryTest checkAnswer(join2, df) assert( collect(join2.queryExecution.executedPlan) { - case e: ShuffleExchangeExec => true }.size == 1) -+ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => true }.size == 1) ++ case _: ShuffleExchangeLike => true }.size == 1) assert( collect(join2.queryExecution.executedPlan) { case e: BroadcastExchangeExec => true }.size === 1) -@@ -2863,6 +2864,7 @@ class DataFrameSuite extends QueryTest +@@ -2862,7 +2862,7 @@ class DataFrameSuite extends QueryTest + // Assert that no extra shuffle introduced by cogroup. val exchanges = collect(df3.queryExecution.executedPlan) { - case h: ShuffleExchangeExec => h -+ case h: CometShuffleExchangeExec => h +- case h: ShuffleExchangeExec => h ++ case h: ShuffleExchangeLike => h } assert(exchanges.size == 2) } -@@ -3311,7 +3313,8 @@ class DataFrameSuite extends QueryTest +@@ -3311,7 +3311,8 @@ class DataFrameSuite extends QueryTest assert(df2.isLocal) } @@ -316,22 +330,24 @@ index 9ddb4abe98b..2bd28d4041d 100644 sql( """ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala -index 7dec558f8df..064cf6d4d97 100644 +index 7dec558f8df..840dda15033 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala -@@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} - import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema - import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} +@@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} import org.apache.spark.sql.catalyst.util.sideBySide -+import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec, SQLExecution} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper - import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} -@@ -2255,6 +2256,7 @@ class DatasetSuite extends QueryTest +-import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} ++import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.streaming.MemoryStream + import org.apache.spark.sql.expressions.UserDefinedFunction + import org.apache.spark.sql.functions._ +@@ -2254,7 +2254,7 @@ class DatasetSuite extends QueryTest + // Assert that no extra shuffle introduced by cogroup. val exchanges = collect(df3.queryExecution.executedPlan) { - case h: ShuffleExchangeExec => h -+ case c: CometShuffleExchangeExec => c +- case h: ShuffleExchangeExec => h ++ case h: ShuffleExchangeLike => h } assert(exchanges.size == 2) } @@ -542,19 +558,18 @@ index 1792b4c32eb..1616e6f39bd 100644 assert(shuffleMergeJoins.size == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -index 5125708be32..cff6252a36b 100644 +index 5125708be32..210ab4f3ce1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -@@ -29,6 +29,8 @@ import org.apache.spark.sql.catalyst.TableIdentifier +@@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, GenericRow, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.comet._ -+import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} -@@ -739,7 +741,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -739,7 +740,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } } @@ -564,7 +579,7 @@ index 5125708be32..cff6252a36b 100644 withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "0", SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> "1") { -@@ -1114,9 +1117,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1114,9 +1116,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) .groupBy($"k1").count() .queryExecution.executedPlan @@ -574,11 +589,11 @@ index 5125708be32..cff6252a36b 100644 // No extra shuffle before aggregate - assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 2) + assert(collect(plan) { -+ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => true }.size === 2) ++ case _: ShuffleExchangeLike => true }.size === 2) }) } -@@ -1133,10 +1138,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1133,10 +1137,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) .queryExecution .executedPlan @@ -592,7 +607,7 @@ index 5125708be32..cff6252a36b 100644 }) // Test shuffled hash join -@@ -1146,10 +1152,13 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1146,10 +1151,13 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) .queryExecution .executedPlan @@ -609,7 +624,7 @@ index 5125708be32..cff6252a36b 100644 }) } -@@ -1240,12 +1249,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1240,12 +1248,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan inputDFs.foreach { case (df1, df2, joinExprs) => val smjDF = df1.join(df2.hint("SHUFFLE_MERGE"), joinExprs, "full") assert(collect(smjDF.queryExecution.executedPlan) { @@ -624,17 +639,17 @@ index 5125708be32..cff6252a36b 100644 // Same result between shuffled hash join and sort merge join checkAnswer(shjDF, smjResult) } -@@ -1340,7 +1349,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1340,7 +1348,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) // Have shuffle before aggregation - assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 1) + assert(collect(plan) { -+ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => true }.size === 1) ++ case _: ShuffleExchangeLike => true }.size === 1) } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1369,9 +1379,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1369,9 +1378,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -649,7 +664,7 @@ index 5125708be32..cff6252a36b 100644 } // Test output ordering is not preserved -@@ -1380,9 +1393,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1380,9 +1392,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val selectExpr = "/*+ BROADCAST(left_t) */ k1 as k0" val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -664,13 +679,13 @@ index 5125708be32..cff6252a36b 100644 } // Test singe partition -@@ -1392,7 +1408,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1392,7 +1407,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |FROM range(0, 10, 1, 1) t1 FULL OUTER JOIN range(0, 10, 1, 1) t2 |""".stripMargin) val plan = fullJoinDF.queryExecution.executedPlan - assert(collect(plan) { case _: ShuffleExchangeExec => true}.size == 1) + assert(collect(plan) { -+ case _: ShuffleExchangeExec | _: CometShuffleExchangeExec => true}.size == 1) ++ case _: ShuffleExchangeLike => true}.size == 1) checkAnswer(fullJoinDF, Row(100)) } } @@ -702,19 +717,23 @@ index 525d97e4998..8a3e7457618 100644 sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC").collect() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala -index 3cfda19134a..278bb1060c4 100644 +index 3cfda19134a..7590b808def 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala -@@ -21,6 +21,8 @@ import scala.collection.mutable.ArrayBuffer +@@ -21,10 +21,11 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Join, LogicalPlan, Project, Sort, Union} +import org.apache.spark.sql.comet.CometScanExec -+import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecution} import org.apache.spark.sql.execution.datasources.FileScanRDD -@@ -1543,6 +1545,12 @@ class SubquerySuite extends QueryTest +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike + import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession +@@ -1543,6 +1544,12 @@ class SubquerySuite extends QueryTest fs.inputRDDs().forall( _.asInstanceOf[FileScanRDD].filePartitions.forall( _.files.forall(_.urlEncodedPath.contains("p=0")))) @@ -727,47 +746,58 @@ index 3cfda19134a..278bb1060c4 100644 case _ => false }) } -@@ -2109,6 +2117,7 @@ class SubquerySuite extends QueryTest +@@ -2108,7 +2115,7 @@ class SubquerySuite extends QueryTest + df.collect() val exchanges = collect(df.queryExecution.executedPlan) { - case s: ShuffleExchangeExec => s -+ case s: CometShuffleExchangeExec => s +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s } assert(exchanges.size === 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala -index 02990a7a40d..2f89e7eb391 100644 +index 02990a7a40d..bddf5e1ccc2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala -@@ -24,6 +24,8 @@ import test.org.apache.spark.sql.connector._ +@@ -24,6 +24,7 @@ import test.org.apache.spark.sql.connector._ import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.comet.CometSortExec -+import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.connector.catalog.{PartitionInternalRow, SupportsRead, Table, TableCapability, TableProvider} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.expressions.{Expression, FieldReference, Literal, NamedReference, NullOrdering, SortDirection, SortOrder, Transform} -@@ -269,12 +271,14 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS +@@ -33,7 +34,7 @@ import org.apache.spark.sql.connector.read.partitioning.{KeyGroupedPartitioning, + import org.apache.spark.sql.execution.SortExec + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, DataSourceV2ScanRelation} +-import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} ++import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector + import org.apache.spark.sql.expressions.Window + import org.apache.spark.sql.functions._ +@@ -268,13 +269,13 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS + val groupByColJ = df.groupBy($"j").agg(sum($"i")) checkAnswer(groupByColJ, Seq(Row(2, 8), Row(4, 2), Row(6, 5))) assert(collectFirst(groupByColJ.queryExecution.executedPlan) { - case e: ShuffleExchangeExec => e -+ case c: CometShuffleExchangeExec => c +- case e: ShuffleExchangeExec => e ++ case e: ShuffleExchangeLike => e }.isDefined) val groupByIPlusJ = df.groupBy($"i" + $"j").agg(count("*")) checkAnswer(groupByIPlusJ, Seq(Row(5, 2), Row(6, 2), Row(8, 1), Row(9, 1))) assert(collectFirst(groupByIPlusJ.queryExecution.executedPlan) { - case e: ShuffleExchangeExec => e -+ case c: CometShuffleExchangeExec => c +- case e: ShuffleExchangeExec => e ++ case e: ShuffleExchangeLike => e }.isDefined) } } -@@ -335,9 +339,11 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS +@@ -334,10 +335,11 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS + val (shuffleExpected, sortExpected) = groupByExpects assert(collectFirst(groupBy.queryExecution.executedPlan) { - case e: ShuffleExchangeExec => e -+ case c: CometShuffleExchangeExec => c +- case e: ShuffleExchangeExec => e ++ case e: ShuffleExchangeLike => e }.isDefined === shuffleExpected) assert(collectFirst(groupBy.queryExecution.executedPlan) { case e: SortExec => e @@ -775,11 +805,12 @@ index 02990a7a40d..2f89e7eb391 100644 }.isDefined === sortExpected) } -@@ -353,9 +359,11 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS +@@ -352,10 +354,11 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS + val (shuffleExpected, sortExpected) = windowFuncExpects assert(collectFirst(windowPartByColIOrderByColJ.queryExecution.executedPlan) { - case e: ShuffleExchangeExec => e -+ case c: CometShuffleExchangeExec => c +- case e: ShuffleExchangeExec => e ++ case e: ShuffleExchangeLike => e }.isDefined === shuffleExpected) assert(collectFirst(windowPartByColIOrderByColJ.queryExecution.executedPlan) { case e: SortExec => e @@ -813,28 +844,27 @@ index cfc8b2cc845..c6fcfd7bd08 100644 } finally { spark.listenerManager.unregister(listener) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala -index cf76f6ca32c..8a7c2b894ad 100644 +index cf76f6ca32c..f454128af06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala -@@ -22,6 +22,8 @@ import org.apache.spark.sql.{DataFrame, Row} +@@ -22,6 +22,7 @@ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Literal, TransformExpression} import org.apache.spark.sql.catalyst.plans.physical +import org.apache.spark.sql.comet.CometSortMergeJoinExec -+import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.InMemoryTableCatalog import org.apache.spark.sql.connector.catalog.functions._ -@@ -31,7 +33,7 @@ import org.apache.spark.sql.connector.expressions.Expressions._ +@@ -31,7 +32,7 @@ import org.apache.spark.sql.connector.expressions.Expressions._ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -+import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} ++import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf._ -@@ -279,13 +281,15 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { +@@ -279,13 +280,14 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { Row("bbb", 20, 250.0), Row("bbb", 20, 350.0), Row("ccc", 30, 400.50))) } @@ -846,8 +876,8 @@ index cf76f6ca32c..8a7c2b894ad 100644 + case c: CometSortMergeJoinExec => c.originalPlan }.flatMap(smj => collect(smj) { - case s: ShuffleExchangeExec => s -+ case c: CometShuffleExchangeExec => c +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s }) } @@ -1072,7 +1102,7 @@ index ac710c32296..e163c1a6a76 100644 val df = spark.read.parquet(path).selectExpr(projection: _*) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -index 593bd7bb4ba..2518d715154 100644 +index 593bd7bb4ba..7ad55e3ab20 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -26,9 +26,11 @@ import org.scalatest.time.SpanSugar._ @@ -1143,15 +1173,16 @@ index 593bd7bb4ba..2518d715154 100644 assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) } -@@ -185,6 +197,7 @@ class AdaptiveQueryExecSuite +@@ -184,7 +196,7 @@ class AdaptiveQueryExecSuite + val plan = df.queryExecution.executedPlan assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect { - case s: ShuffleExchangeExec => s -+ case c: CometShuffleExchangeExec => c +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s } assert(shuffle.size == 1) assert(shuffle(0).outputPartitioning.numPartitions == numPartition) -@@ -200,7 +213,8 @@ class AdaptiveQueryExecSuite +@@ -200,7 +212,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1161,7 +1192,7 @@ index 593bd7bb4ba..2518d715154 100644 } } -@@ -227,7 +241,8 @@ class AdaptiveQueryExecSuite +@@ -227,7 +240,8 @@ class AdaptiveQueryExecSuite } } @@ -1171,7 +1202,7 @@ index 593bd7bb4ba..2518d715154 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", -@@ -259,7 +274,8 @@ class AdaptiveQueryExecSuite +@@ -259,7 +273,8 @@ class AdaptiveQueryExecSuite } } @@ -1181,7 +1212,7 @@ index 593bd7bb4ba..2518d715154 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", -@@ -273,7 +289,8 @@ class AdaptiveQueryExecSuite +@@ -273,7 +288,8 @@ class AdaptiveQueryExecSuite val localReads = collect(adaptivePlan) { case read: AQEShuffleReadExec if read.isLocalRead => read } @@ -1191,7 +1222,7 @@ index 593bd7bb4ba..2518d715154 100644 val localShuffleRDD0 = localReads(0).execute().asInstanceOf[ShuffledRowRDD] val localShuffleRDD1 = localReads(1).execute().asInstanceOf[ShuffledRowRDD] // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 -@@ -322,7 +339,7 @@ class AdaptiveQueryExecSuite +@@ -322,7 +338,7 @@ class AdaptiveQueryExecSuite } } @@ -1200,7 +1231,7 @@ index 593bd7bb4ba..2518d715154 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -337,7 +354,7 @@ class AdaptiveQueryExecSuite +@@ -337,7 +353,7 @@ class AdaptiveQueryExecSuite } } @@ -1209,7 +1240,7 @@ index 593bd7bb4ba..2518d715154 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -353,7 +370,7 @@ class AdaptiveQueryExecSuite +@@ -353,7 +369,7 @@ class AdaptiveQueryExecSuite } } @@ -1218,7 +1249,7 @@ index 593bd7bb4ba..2518d715154 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -398,7 +415,7 @@ class AdaptiveQueryExecSuite +@@ -398,7 +414,7 @@ class AdaptiveQueryExecSuite } } @@ -1227,7 +1258,7 @@ index 593bd7bb4ba..2518d715154 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -443,7 +460,7 @@ class AdaptiveQueryExecSuite +@@ -443,7 +459,7 @@ class AdaptiveQueryExecSuite } } @@ -1236,7 +1267,7 @@ index 593bd7bb4ba..2518d715154 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { -@@ -508,7 +525,7 @@ class AdaptiveQueryExecSuite +@@ -508,7 +524,7 @@ class AdaptiveQueryExecSuite } } @@ -1245,7 +1276,7 @@ index 593bd7bb4ba..2518d715154 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { -@@ -539,7 +556,9 @@ class AdaptiveQueryExecSuite +@@ -539,7 +555,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1256,7 +1287,7 @@ index 593bd7bb4ba..2518d715154 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) -@@ -560,7 +579,9 @@ class AdaptiveQueryExecSuite +@@ -560,7 +578,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) @@ -1267,7 +1298,7 @@ index 593bd7bb4ba..2518d715154 100644 // Even with local shuffle read, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.isEmpty) -@@ -569,7 +590,8 @@ class AdaptiveQueryExecSuite +@@ -569,7 +589,8 @@ class AdaptiveQueryExecSuite } } @@ -1277,7 +1308,7 @@ index 593bd7bb4ba..2518d715154 100644 withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", -@@ -664,7 +686,8 @@ class AdaptiveQueryExecSuite +@@ -664,7 +685,8 @@ class AdaptiveQueryExecSuite val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) // There is still a SMJ, and its two shuffles can't apply local read. @@ -1287,7 +1318,7 @@ index 593bd7bb4ba..2518d715154 100644 } } -@@ -786,7 +809,8 @@ class AdaptiveQueryExecSuite +@@ -786,7 +808,8 @@ class AdaptiveQueryExecSuite } } @@ -1297,7 +1328,7 @@ index 593bd7bb4ba..2518d715154 100644 Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { joinHint => def getJoinNode(plan: SparkPlan): Seq[ShuffledJoin] = if (joinHint == "SHUFFLE_MERGE") { findTopLevelSortMergeJoin(plan) -@@ -1004,7 +1028,8 @@ class AdaptiveQueryExecSuite +@@ -1004,7 +1027,8 @@ class AdaptiveQueryExecSuite } } @@ -1307,15 +1338,16 @@ index 593bd7bb4ba..2518d715154 100644 withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT key FROM testData GROUP BY key") -@@ -1600,6 +1625,7 @@ class AdaptiveQueryExecSuite +@@ -1599,7 +1623,7 @@ class AdaptiveQueryExecSuite + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") assert(collect(adaptivePlan) { - case s: ShuffleExchangeExec => s -+ case c: CometShuffleExchangeExec => c +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s }.length == 1) } } -@@ -1679,7 +1705,8 @@ class AdaptiveQueryExecSuite +@@ -1679,7 +1703,8 @@ class AdaptiveQueryExecSuite } } @@ -1325,7 +1357,7 @@ index 593bd7bb4ba..2518d715154 100644 def hasRepartitionShuffle(plan: SparkPlan): Boolean = { find(plan) { case s: ShuffleExchangeLike => -@@ -1864,6 +1891,9 @@ class AdaptiveQueryExecSuite +@@ -1864,6 +1889,9 @@ class AdaptiveQueryExecSuite def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { assert(collect(ds.queryExecution.executedPlan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1335,7 +1367,7 @@ index 593bd7bb4ba..2518d715154 100644 }.size == 1) ds.collect() val plan = ds.queryExecution.executedPlan -@@ -1872,6 +1902,9 @@ class AdaptiveQueryExecSuite +@@ -1872,6 +1900,9 @@ class AdaptiveQueryExecSuite }.isEmpty) assert(collect(plan) { case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s @@ -1345,7 +1377,7 @@ index 593bd7bb4ba..2518d715154 100644 }.size == 1) checkAnswer(ds, testData) } -@@ -2028,7 +2061,8 @@ class AdaptiveQueryExecSuite +@@ -2028,7 +2059,8 @@ class AdaptiveQueryExecSuite } } @@ -1355,7 +1387,7 @@ index 593bd7bb4ba..2518d715154 100644 withTempView("t1", "t2") { def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { Seq("100", "100000").foreach { size => -@@ -2114,7 +2148,8 @@ class AdaptiveQueryExecSuite +@@ -2114,7 +2146,8 @@ class AdaptiveQueryExecSuite } } @@ -1365,23 +1397,25 @@ index 593bd7bb4ba..2518d715154 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", -@@ -2214,6 +2249,7 @@ class AdaptiveQueryExecSuite +@@ -2213,7 +2246,7 @@ class AdaptiveQueryExecSuite + runAdaptiveAndVerifyResult(s"SELECT $repartition key1 FROM skewData1 " + s"JOIN skewData2 ON key1 = key2 GROUP BY key1") val shuffles1 = collect(adaptive1) { - case s: ShuffleExchangeExec => s -+ case c: CometShuffleExchangeExec => c.originalPlan +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s } assert(shuffles1.size == 3) // shuffles1.head is the top-level shuffle under the Aggregate operator -@@ -2227,6 +2263,7 @@ class AdaptiveQueryExecSuite +@@ -2226,7 +2259,7 @@ class AdaptiveQueryExecSuite + runAdaptiveAndVerifyResult(s"SELECT $repartition key1 FROM skewData1 " + s"JOIN skewData2 ON key1 = key2") val shuffles2 = collect(adaptive2) { - case s: ShuffleExchangeExec => s -+ case c: CometShuffleExchangeExec => c.originalPlan +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s } if (hasRequiredDistribution) { assert(shuffles2.size == 3) -@@ -2260,7 +2297,8 @@ class AdaptiveQueryExecSuite +@@ -2260,7 +2293,8 @@ class AdaptiveQueryExecSuite } } @@ -1391,7 +1425,7 @@ index 593bd7bb4ba..2518d715154 100644 CostEvaluator.instantiate( classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, spark.sparkContext.getConf) intercept[IllegalArgumentException] { -@@ -2404,6 +2442,7 @@ class AdaptiveQueryExecSuite +@@ -2404,6 +2438,7 @@ class AdaptiveQueryExecSuite val (_, adaptive) = runAdaptiveAndVerifyResult(query) assert(adaptive.collect { case sort: SortExec => sort @@ -1399,7 +1433,7 @@ index 593bd7bb4ba..2518d715154 100644 }.size == 1) val read = collect(adaptive) { case read: AQEShuffleReadExec => read -@@ -2421,7 +2460,8 @@ class AdaptiveQueryExecSuite +@@ -2421,7 +2456,8 @@ class AdaptiveQueryExecSuite } } @@ -1409,19 +1443,21 @@ index 593bd7bb4ba..2518d715154 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", -@@ -2534,6 +2574,7 @@ class AdaptiveQueryExecSuite +@@ -2533,7 +2569,7 @@ class AdaptiveQueryExecSuite + runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + "JOIN skewData3 ON value2 = value3") val shuffles1 = collect(adaptive1) { - case s: ShuffleExchangeExec => s -+ case c: CometShuffleExchangeExec => c +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s } assert(shuffles1.size == 4) val smj1 = findTopLevelSortMergeJoin(adaptive1) -@@ -2545,6 +2586,7 @@ class AdaptiveQueryExecSuite +@@ -2544,7 +2580,7 @@ class AdaptiveQueryExecSuite + runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + "JOIN skewData3 ON value1 = value3") val shuffles2 = collect(adaptive2) { - case s: ShuffleExchangeExec => s -+ case c: CometShuffleExchangeExec => c +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s } assert(shuffles2.size == 4) val smj2 = findTopLevelSortMergeJoin(adaptive2) @@ -1854,21 +1890,24 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 266bb343526..b33bb677f0d 100644 +index 266bb343526..a426d8396be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -@@ -24,7 +24,9 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec +@@ -24,10 +24,11 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.{FileSourceScanExec, SortExec, SparkPlan} +import org.apache.spark.sql.comet._ -+import org.apache.spark.sql.comet.execution.shuffle._ +import org.apache.spark.sql.execution.{ColumnarToRowExec, FileSourceScanExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, DisableAdaptiveExecution} import org.apache.spark.sql.execution.datasources.BucketingUtils - import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -@@ -101,12 +103,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.joins.SortMergeJoinExec + import org.apache.spark.sql.functions._ + import org.apache.spark.sql.internal.SQLConf +@@ -101,12 +102,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -1891,7 +1930,7 @@ index 266bb343526..b33bb677f0d 100644 // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -155,7 +165,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -155,7 +164,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -1901,7 +1940,7 @@ index 266bb343526..b33bb677f0d 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -451,28 +462,46 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -451,28 +461,44 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -1934,13 +1973,11 @@ index 266bb343526..b33bb677f0d 100644 // check existence of shuffle assert( - joinOperator.left.exists(_.isInstanceOf[ShuffleExchangeExec]) == shuffleLeft, -+ joinOperator.left.exists(op => op.isInstanceOf[ShuffleExchangeExec] || -+ op.isInstanceOf[CometShuffleExchangeExec]) == shuffleLeft, ++ joinOperator.left.exists(op => op.isInstanceOf[ShuffleExchangeLike]) == shuffleLeft, s"expected shuffle in plan to be $shuffleLeft but found\n${joinOperator.left}") assert( - joinOperator.right.exists(_.isInstanceOf[ShuffleExchangeExec]) == shuffleRight, -+ joinOperator.right.exists(op => op.isInstanceOf[ShuffleExchangeExec] || -+ op.isInstanceOf[CometShuffleExchangeExec]) == shuffleRight, ++ joinOperator.right.exists(op => op.isInstanceOf[ShuffleExchangeLike]) == shuffleRight, s"expected shuffle in plan to be $shuffleRight but found\n${joinOperator.right}") // check existence of sort @@ -1956,7 +1993,7 @@ index 266bb343526..b33bb677f0d 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -835,11 +864,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -835,11 +861,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -1970,14 +2007,13 @@ index 266bb343526..b33bb677f0d 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -1026,15 +1055,24 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1026,15 +1052,23 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti expectedNumShuffles: Int, expectedCoalescedNumBuckets: Option[Int]): Unit = { val plan = sql(query).queryExecution.executedPlan - val shuffles = plan.collect { case s: ShuffleExchangeExec => s } + val shuffles = plan.collect { -+ case s: ShuffleExchangeExec => s -+ case s: CometShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s + } assert(shuffles.length == expectedNumShuffles)